You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "satishd (via GitHub)" <gi...@apache.org> on 2023/02/18 11:32:02 UTC

[GitHub] [kafka] satishd opened a new pull request, #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

satishd opened a new pull request, #13275:
URL: https://github.com/apache/kafka/pull/13275

   KAFKA-14522 Rewrite/Move of RemoteIndexCache to the storage module.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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


[GitHub] [kafka] junrao commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1255005974


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {

Review Comment:
   This and the next two methods are only used for tests. Should we expose them at the package level?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.debug("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");

Review Comment:
   ShutdownableThread already logs the shutdown.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.debug("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");
+                        }
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String fileNameStr = fileNamePath.toString();
+                String name = fileNameStr.substring(0, fileNameStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                long offset = Long.parseLong(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            Uuid cacheKey = remoteLogSegmentMetadata.remoteLogSegmentId().id();
+            return internalCache.get(cacheKey, (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TransactionIndex index = new TransactionIndex(startOffset, file);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());

Review Comment:
   Does this need to be done between `initiateShutdown` and `awaitShutdown`? Typically, this is only needed if you want to make sure sth happens during the shutdown. If not, the simpler approach is to make a single call `shutdown`.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.debug("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");
+                        }
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {

Review Comment:
   Should we clean up the tmp file too?



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


[GitHub] [kafka] jeqo commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1258867599


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ * This class is thread safe.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    // Visible for testing
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index file {}", indexFile.getPath(), ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public static class Entry implements AutoCloseable {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;

Review Comment:
   Should this be optional?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ * This class is thread safe.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    // Visible for testing
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index file {}", indexFile.getPath(), ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);

Review Comment:
   Same here, what if this returns `RemoteResourceNotFound`?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ * This class is thread safe.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    // Visible for testing
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);

Review Comment:
   What if this segment does not contain transaction index as it's optional? Should the condition to create entries consider this alternative?



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257709493


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.debug("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");

Review Comment:
   This comment is very specific to the interrupted scenario. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257708611


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());

Review Comment:
   I made the change in this PR, we can have the respective test in a followup PR. Added https://issues.apache.org/jira/browse/KAFKA-15169 to followup on the test. 



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


[GitHub] [kafka] showuon commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1621237274

   Will take a look this week or next week. Thanks.


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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1123037354


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);

Review Comment:
   Good catch! It can be KafkaException like it is done in other places in the same method. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1123369985


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {

Review Comment:
   Right, it was not a deliberate change.



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


[GitHub] [kafka] junrao commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1128778509


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize / 2, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));

Review Comment:
   https://github.com/apache/kafka/pull/13342/files pointed out that we should parse long instead of int here. Should we just fix it here?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {

Review Comment:
   If there is an IOException, we want to propagate it to LogDirFailureChannel instead of hiding it. Perhaps we could avoid using `foreach` lamda in this case?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;
+                        try {
+                            index = new TransactionIndex(startOffset, file);
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        index.sanityCheck();
+                        return index;
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+    }
+
+    public void close() {
+        closed = true;
+        try {
+            cleanerThread.shutdown();
+        } catch (InterruptedException e) {
+            // ignore interrupted exception
+        }
+
+        // Close all the opened indexes.
+        synchronized (lock) {
+            entries.values().forEach(Entry::close);
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(entries);
+    }
+
+    public static class Entry {
+
+        public final OffsetIndex offsetIndex;
+        public final TimeIndex timeIndex;
+        public final TransactionIndex txnIndex;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        private boolean markedForCleanup = false;
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        public OffsetPosition lookupOffset(long targetOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+
+            try {
+                LogFileUtils.tryAll(Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                }));
+            } catch (Exception e) {

Review Comment:
   If we get an IOException, we want to propagate it. If we get any other Exception, it seems that we should convert it to KafkaException?



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1259130451


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ * This class is thread safe.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    // Visible for testing
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index file {}", indexFile.getPath(), ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);

Review Comment:
   Please take a look at the [comment](https://github.com/apache/kafka/pull/13275#issuecomment-1630042350).



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1123045997


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;
+                        try {
+                            index = new TransactionIndex(startOffset, file);
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        index.sanityCheck();
+                        return index;
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+    }
+
+    public void close() {
+        closed = true;
+        try {
+            cleanerThread.shutdown();
+        } catch (InterruptedException e) {
+            // ignore interrupted exception
+        }
+
+        // Close all the opened indexes.
+        synchronized (lock) {
+            entries.values().forEach(Entry::close);
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(entries);
+    }
+
+    public static class Entry {
+
+        public final OffsetIndex offsetIndex;
+        public final TimeIndex timeIndex;
+        public final TransactionIndex txnIndex;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        private boolean markedForCleanup = false;
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        public OffsetPosition lookupOffset(long targetOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));

Review Comment:
   Good catch! It was missed while converting the code. 



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


[GitHub] [kafka] junrao commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1119167627


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+

Review Comment:
   Could we remove the extra new line?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {

Review Comment:
   Hmm, should we convert IOException to KafkaException?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);

Review Comment:
   The constructor declares throwing IOException. Why do we need to convert IOException to KafkaException? Ditto in other methods like `init()`?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;
+                        try {
+                            index = new TransactionIndex(startOffset, file);
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        index.sanityCheck();
+                        return index;
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+    }
+
+    public void close() {
+        closed = true;
+        try {
+            cleanerThread.shutdown();
+        } catch (InterruptedException e) {
+            // ignore interrupted exception
+        }
+
+        // Close all the opened indexes.
+        synchronized (lock) {
+            entries.values().forEach(Entry::close);
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(entries);
+    }
+
+    public static class Entry {
+
+        public final OffsetIndex offsetIndex;
+        public final TimeIndex timeIndex;
+        public final TransactionIndex txnIndex;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        private boolean markedForCleanup = false;
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        public OffsetPosition lookupOffset(long targetOffset) throws IOException {

Review Comment:
   This doesn't seem to throw IOException. Ditto for lookupTimestamp.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;
+                        try {
+                            index = new TransactionIndex(startOffset, file);
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        index.sanityCheck();
+                        return index;
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+    }
+
+    public void close() {
+        closed = true;
+        try {
+            cleanerThread.shutdown();
+        } catch (InterruptedException e) {
+            // ignore interrupted exception
+        }
+
+        // Close all the opened indexes.
+        synchronized (lock) {
+            entries.values().forEach(Entry::close);
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(entries);
+    }
+
+    public static class Entry {
+
+        public final OffsetIndex offsetIndex;
+        public final TimeIndex timeIndex;
+        public final TransactionIndex txnIndex;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        private boolean markedForCleanup = false;
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        public OffsetPosition lookupOffset(long targetOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+
+            try {
+                LogFileUtils.tryAll(Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                }));
+            } catch (Exception e) {

Review Comment:
   Should we propagate IOException?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);

Review Comment:
   Hmm, should this be KafkaException?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {

Review Comment:
   The scala code used `maxSize/2`. Is this change expected?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;

Review Comment:
   Is this line needed?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;
+                        try {
+                            index = new TransactionIndex(startOffset, file);
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        index.sanityCheck();
+                        return index;
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+    }
+
+    public void close() {
+        closed = true;
+        try {
+            cleanerThread.shutdown();
+        } catch (InterruptedException e) {
+            // ignore interrupted exception
+        }
+
+        // Close all the opened indexes.
+        synchronized (lock) {
+            entries.values().forEach(Entry::close);
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(entries);
+    }
+
+    public static class Entry {
+
+        public final OffsetIndex offsetIndex;
+        public final TimeIndex timeIndex;
+        public final TransactionIndex txnIndex;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        private boolean markedForCleanup = false;
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        public OffsetPosition lookupOffset(long targetOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));

Review Comment:
   Do we need to do the same for timestamp index too?



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


[GitHub] [kafka] satishd commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1630042350

   > Just a few comments related to TxnIndex potentially being optional.
   
   @jeqo  There is already [KAFKA-14993](https://issues.apache.org/jira/browse/KAFKA-14993) to address that. @kamalcph was working on that. I did not want to add those changes to this PR as it is tracked separately. 
   
   


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


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257919162


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,666 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {

Review Comment:
   nit
   
   `// visible for testing`
   



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,666 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {

Review Comment:
   Annotation `@threadsafe` (similar to scala code) since it helps the code reader or the user of the class understand the threading model expected by this class. In absence of that please add comment on the javadoc that this class is thread safe.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,666 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index file {}", indexFile.getPath(), ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public static class Entry implements AutoCloseable {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;
+
+        // This lock is used to synchronize cleanup methods and read methods. This ensures that cleanup (which changes the
+        // underlying files of the index) isn't performed while a read is in-progress for the entry. This is required in
+        // addition to using the thread safe cache because, while the thread safety of the cache ensures that we can read
+        // entries concurrently, it does not ensure that we won't mutate underlying files belonging to an entry.
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        private boolean cleanStarted = false;
+
+        private boolean markedForCleanup = false;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        public OffsetIndex offsetIndex() {
+            return offsetIndex;
+        }
+
+        public TimeIndex timeIndex() {
+            return timeIndex;
+        }
+
+        public TransactionIndex txnIndex() {

Review Comment:
   should be under readlock (or visible for testing)



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


[GitHub] [kafka] junrao commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1256287178


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -16,6 +16,9 @@
  */
 package org.apache.kafka.storage.internals.log;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;

Review Comment:
   Thanks for the explanation, Divij. This change seems fine to me then.



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


[GitHub] [kafka] satishd commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1619597800

   @junrao @showuon   Addressed the review comments with the latest commits. It also includes the recent changes on `RemoteIndexCache` in trunk. 


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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257709316


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {

Review Comment:
   Tests are written at different package level in core module. We can revisit this once the tests are also moved to java and the respective module. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1111260854


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    // All the below suffixes will be replaced with UnifiedLog once it is moved to storage module.
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Created RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCLeanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCLeanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            Files.deleteIfExists(offsetIndexFile.toPath());
+                            Files.deleteIfExists(timestampIndexFile.toPath());
+                            Files.deleteIfExists(txnIndexFile.toPath());

Review Comment:
   Sure. It does not make sure each index is deleted but it tries at best effort to execute all the deletions even when one of them is failed in between. Updated with the latest commit.  



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


[GitHub] [kafka] ijuma commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1130403919


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogFileUtils.java:
##########
@@ -187,4 +188,25 @@ public static Long offsetFromFile(File file) {
         return offsetFromFileName(file.getName());
     }
 
+    /**
+     * Invokes every function in `all` even if one or more functions throws an exception.
+     * If any of the functions throws an exception, the first one will be rethrown at the end with subsequent exceptions
+     * added as suppressed exceptions.
+     */
+    public static void tryAll(List<StorageAction<Void, Exception>> all) throws Exception {

Review Comment:
   A bit odd to have this method here. This class is meant to have file related utilities and this is not that.



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1121168324


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogFileUtils.java:
##########
@@ -31,6 +32,21 @@ public final class LogFileUtils {
      */
     public static final String DELETED_FILE_SUFFIX = ".deleted";
 
+    /**
+     * Suffix of an offset index file
+     */
+    public static final String INDEX_FILE_SUFFIX = ".index";

Review Comment:
   Right, that is the plan.



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1123035787


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {

Review Comment:
   I did not get the suggestion here. We are catching `Exception` and throwing `KafkaException`. We can not throw IOException here as it is inside`stream.forEach()` lambda function.



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


[GitHub] [kafka] showuon commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1121043658


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogFileUtils.java:
##########
@@ -31,6 +32,21 @@ public final class LogFileUtils {
      */
     public static final String DELETED_FILE_SUFFIX = ".deleted";
 
+    /**
+     * Suffix of an offset index file
+     */
+    public static final String INDEX_FILE_SUFFIX = ".index";

Review Comment:
   Looks like it'll get conflicted with another PR. But I think you'll resolve them later.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;
+                        try {
+                            index = new TransactionIndex(startOffset, file);
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        index.sanityCheck();
+                        return index;
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+    }
+
+    public void close() {
+        closed = true;
+        try {
+            cleanerThread.shutdown();
+        } catch (InterruptedException e) {
+            // ignore interrupted exception
+        }
+
+        // Close all the opened indexes.
+        synchronized (lock) {
+            entries.values().forEach(Entry::close);
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(entries);
+    }
+
+    public static class Entry {
+
+        public final OffsetIndex offsetIndex;
+        public final TimeIndex timeIndex;
+        public final TransactionIndex txnIndex;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        private boolean markedForCleanup = false;
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        public OffsetPosition lookupOffset(long targetOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+
+            try {
+                LogFileUtils.tryAll(Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                }));
+            } catch (Exception e) {
+                throw new KafkaException(e);
+            }
+        }
+
+        public void close() {
+            Arrays.asList(offsetIndex, timeIndex).forEach(index -> {
+                try {
+                    index.close();
+                } catch (Exception e) {
+                    // ignore exception.
+                }
+            });

Review Comment:
   What's the difference between this close and below `closeQuietly`? Could we replace with `closeQuietly`?



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


[GitHub] [kafka] satishd commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1629214523

   Thanks @divijvaidya for the latest review. Addressed them with the latest commit.


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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257700736


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {

Review Comment:
   It was meant for testing but it returned unmodifiable collection. This method is no more used with the latest changes. 



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


[GitHub] [kafka] showuon commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1255129114


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -16,6 +16,9 @@
  */
 package org.apache.kafka.storage.internals.log;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;

Review Comment:
   Caffeine is added in this [PR](https://github.com/apache/kafka/pull/13850) to improve concurrent read performance. We chose it since it has good performance and is also adopted by other big projects, like Cassandra, HBase etc. About the GC issues, I don't think we have discussed about it. cc @divijvaidya , since you've done some research on Caffeine, do you have any comment to the GC issue?



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1259126161


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ * This class is thread safe.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    // Visible for testing
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);

Review Comment:
   Currently, it creates an empty file. There is already https://issues.apache.org/jira/browse/KAFKA-14993 to address that.



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


[GitHub] [kafka] showuon commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1111157501


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    // All the below suffixes will be replaced with UnifiedLog once it is moved to storage module.
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Created RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCLeanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCLeanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            Files.deleteIfExists(offsetIndexFile.toPath());
+                            Files.deleteIfExists(timestampIndexFile.toPath());
+                            Files.deleteIfExists(txnIndexFile.toPath());

Review Comment:
   Should we use `LogFileUtils.tryAll(Arrays.asList(() -> {}` to make sure each index is deleted?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    // All the below suffixes will be replaced with UnifiedLog once it is moved to storage module.
+    private static final String TMP_FILE_SUFFIX = ".tmp";

Review Comment:
   Do you think this comment should be created as another JIRA ticket, instead of putting in the comment?



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


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1255757382


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -16,6 +16,9 @@
  */
 package org.apache.kafka.storage.internals.log;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;

Review Comment:
   Hey Jun
   
   Caffeine is being used for many years in other latency/memory sensitive projects such as Solr, Cassandra.
   
   Solr: It is being used in Apache Solr since 2019[1]. From their release doc[2]:
   > Users are encouraged to transition their cache configurations to use org.apache.solr.search.CaffeineCache as soon as feasible. 
   
   Cassandra: It is being used in Cassandra since 2017 for performance critical caches.
   
   Now, about the weak references, please note that weak & soft references are optional [4], and we don't use weak/soft references in this implementation.. The keys and values are by default stored with strong references. You can see this information at the javadoc [4] the `newBuilder()` method we are using. Quoting from there:
   > Constructs a new Caffeine instance with default settings, including strong keys, strong values, and no automatic eviction of any kind.
   
   Personally, outside of Kafka, I have used this cache in query execution path of database internals and it hasn't caused any problems. Please let me know if you have further question about it's usage. Happy to explain more!
   
   
   [1] https://issues.apache.org/jira/browse/SOLR-8241
   [2] https://solr.apache.org/guide/8_4/solr-upgrade-notes.html#Caches
   [3] https://issues.apache.org/jira/browse/CASSANDRA-10855
   [4] https://www.javadoc.io/doc/com.github.ben-manes.caffeine/caffeine/2.0.3/com/github/benmanes/caffeine/cache/Caffeine.html#newBuilder--



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


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257491521


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(internalCache.asMap());
+    }
+
+    public static class Entry {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;
+
+        // This lock is used to synchronize cleanup methods and read methods. This ensures that cleanup (which changes the
+        // underlying files of the index) isn't performed while a read is in-progress for the entry. This is required in
+        // addition to using the thread safe cache because, while the thread safety of the cache ensures that we can read
+        // entries concurrently, it does not ensure that we won't mutate underlying files belonging to an entry.
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        private boolean cleanStarted = false;
+
+        private boolean markedForCleanup = false;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        public OffsetIndex offsetIndex() {
+            return offsetIndex;
+        }
+
+        public TimeIndex timeIndex() {
+            return timeIndex;
+        }
+
+        public TransactionIndex txnIndex() {
+            return txnIndex;
+        }
+
+        // Visible for testing
+        public boolean isCleanStarted() {
+            return cleanStarted;
+        }
+
+        // Visible for testing
+        public boolean isMarkedForCleanup() {
+            return markedForCleanup;
+        }
+
+        public OffsetPosition lookupOffset(long targetOffset) {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    timeIndex.renameTo(new File(Utils.replaceSuffix(timeIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+            // no-op if clean is done already
+            if (!cleanStarted) {
+                cleanStarted = true;
+
+                List<StorageAction<Void, Exception>> actions = Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                });
+
+                tryAll(actions);
+            }
+        }
+
+        public void close() {

Review Comment:
   nit
   `@Override`



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257709704


##########
build.gradle:
##########
@@ -1704,6 +1704,7 @@ project(':storage') {
     implementation project(':storage:api')
     implementation project(':server-common')
     implementation project(':clients')
+    implementation libs.caffeine

Review Comment:
   This is needed for core/tests module, moved to test dependencies. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257709316


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {

Review Comment:
   Tests are written at different package level in core module. We can revisit this once the tests are also moved to java and the respective module/package. 



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


[GitHub] [kafka] junrao commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1254942551


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -16,6 +16,9 @@
  */
 package org.apache.kafka.storage.internals.log;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;

Review Comment:
   We introduced a new dependency caffeine here. Could you explain why caffeine is chosen and how stable is caffeine? The doc for caffeine mentions the use of weak references. A few years back, we avoided the usage of weak references in a PR because of the poor GC behavior. Have we done any experiments to understand the GC impact?



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1111261087


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    // All the below suffixes will be replaced with UnifiedLog once it is moved to storage module.
+    private static final String TMP_FILE_SUFFIX = ".tmp";

Review Comment:
   This comment is not relevant as those suffixes are already created in LogFileUtils. Removed this comment in the latest commit. 



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


[GitHub] [kafka] satishd commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1452242033

   Thanks @junrao for your review. Addressed your comments inline and/or updated with the latest commit.


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


[GitHub] [kafka] showuon commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1254078224


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,581 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use [[Caffeine]] cache instead of implementing a thread safe LRU cache on our own.

Review Comment:
   This is scala's javadoc format. Please change to java style. 



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,581 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use [[Caffeine]] cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry {}", entry);

Review Comment:
   I will expect this log will be printed very frequently. Should we change to debug or trace level?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,581 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use [[Caffeine]] cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");
+                        }
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String fileNameStr = fileNamePath.toString();
+                String name = fileNameStr.substring(0, fileNameStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            Uuid cacheKey = remoteLogSegmentMetadata.remoteLogSegmentId().id();
+            return internalCache.get(cacheKey, (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TransactionIndex index = new TransactionIndex(startOffset, file);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+

Review Comment:
   Should we set internalCache to null as in scala?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,581 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use [[Caffeine]] cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");
+                        }
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String fileNameStr = fileNamePath.toString();
+                String name = fileNameStr.substring(0, fileNameStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            Uuid cacheKey = remoteLogSegmentMetadata.remoteLogSegmentId().id();
+            return internalCache.get(cacheKey, (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TransactionIndex index = new TransactionIndex(startOffset, file);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(internalCache.asMap());
+    }
+
+    public static class Entry {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;
+
+        // This lock is used to synchronize cleanup methods and read methods. This ensures that cleanup (which changes the
+        // underlying files of the index) isn't performed while a read is in-progress for the entry. This is required in
+        // addition to using the thread safe cache because, while the thread safety of the cache ensures that we can read
+        // entries concurrently, it does not ensure that we won't mutate underlying files belonging to an entry.
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        // Visible for testing
+        private boolean cleanStarted = false;
+
+        // Visible for testing
+        private boolean markedForCleanup = false;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        public OffsetIndex offsetIndex() {
+            return offsetIndex;
+        }
+
+        public TimeIndex timeIndex() {
+            return timeIndex;
+        }
+
+        public TransactionIndex txnIndex() {
+            return txnIndex;
+        }
+
+        // Visible for testing
+        public boolean isCleanStarted() {
+            return cleanStarted;
+        }
+
+        // Visible for testing
+        public boolean isMarkedForCleanup() {
+            return markedForCleanup;
+        }
+
+        public OffsetPosition lookupOffset(long targetOffset) {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    timeIndex.renameTo(new File(Utils.replaceSuffix(timeIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+            // no-op if clean is done already
+            if (!cleanStarted) {
+                cleanStarted = true;
+
+                List<StorageAction<Void, Exception>> actions = Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                });
+
+                tryAll(actions);
+            }
+        }
+
+        public void close() {
+            Utils.closeQuietly(offsetIndex, "OffsetIndex");
+            Utils.closeQuietly(timeIndex, "TimeIndex");
+            Utils.closeQuietly(txnIndex, "TransactionIndex");
+        }
+    }
+
+    /**
+     * Executes each entry in `actions` list even if one or more throws an exception. If any of them throws
+     * an IOException, it will be rethrown and adds all other encountered exceptions as suppressed to that IOException.
+     * Otherwise, it throws KafkaException wrapped with the first exception and the remaining exceptions are added as
+     * suppressed to the KafkaException.
+     *
+     * @param actions actions to be executes
+     * @throws IOException Any IOException encountered while executing those actions.

Review Comment:
   It looks like we also throws KafkaException. Should we add it?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize / 2, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));

Review Comment:
   Will you address this comment?



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


[GitHub] [kafka] showuon commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1255120355


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,581 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use [[Caffeine]] cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");
+                        }
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String fileNameStr = fileNamePath.toString();
+                String name = fileNameStr.substring(0, fileNameStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            Uuid cacheKey = remoteLogSegmentMetadata.remoteLogSegmentId().id();
+            return internalCache.get(cacheKey, (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TransactionIndex index = new TransactionIndex(startOffset, file);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+

Review Comment:
   @divijvaidya , since you wrote the original scala version, any thoughts on this change?



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


[GitHub] [kafka] ijuma commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "ijuma (via GitHub)" <gi...@apache.org>.
ijuma commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1255034976


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -16,6 +16,9 @@
  */
 package org.apache.kafka.storage.internals.log;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;

Review Comment:
   I think this was already there for the Scala code.



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


[GitHub] [kafka] satishd commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1628258188

   Thanks @divijvaidya for the review. Addressed them with inline and/or with the latest commits.


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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257708042


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(internalCache.asMap());
+    }
+
+    public static class Entry {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;
+
+        // This lock is used to synchronize cleanup methods and read methods. This ensures that cleanup (which changes the
+        // underlying files of the index) isn't performed while a read is in-progress for the entry. This is required in
+        // addition to using the thread safe cache because, while the thread safety of the cache ensures that we can read
+        // entries concurrently, it does not ensure that we won't mutate underlying files belonging to an entry.
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        private boolean cleanStarted = false;
+
+        private boolean markedForCleanup = false;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        public OffsetIndex offsetIndex() {
+            return offsetIndex;
+        }
+
+        public TimeIndex timeIndex() {
+            return timeIndex;
+        }
+
+        public TransactionIndex txnIndex() {
+            return txnIndex;
+        }
+
+        // Visible for testing
+        public boolean isCleanStarted() {
+            return cleanStarted;
+        }
+
+        // Visible for testing
+        public boolean isMarkedForCleanup() {
+            return markedForCleanup;
+        }
+
+        public OffsetPosition lookupOffset(long targetOffset) {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    timeIndex.renameTo(new File(Utils.replaceSuffix(timeIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+            // no-op if clean is done already
+            if (!cleanStarted) {
+                cleanStarted = true;
+
+                List<StorageAction<Void, Exception>> actions = Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                });
+
+                tryAll(actions);
+            }
+        }
+
+        public void close() {
+            Utils.closeQuietly(offsetIndex, "OffsetIndex");

Review Comment:
   Good catch! It was unintentional to leave that. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1259130578


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ * This class is thread safe.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    // Visible for testing
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index file {}", indexFile.getPath(), ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public static class Entry implements AutoCloseable {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;

Review Comment:
   Please take a look at the [comment](https://github.com/apache/kafka/pull/13275#issuecomment-1630042350).



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


[GitHub] [kafka] satishd commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1631285075

   A few tests are failed but those are not related to the changes in the PR.


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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1121384368


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogFileUtils.java:
##########
@@ -31,6 +32,21 @@ public final class LogFileUtils {
      */
     public static final String DELETED_FILE_SUFFIX = ".deleted";
 
+    /**
+     * Suffix of an offset index file
+     */
+    public static final String INDEX_FILE_SUFFIX = ".index";

Review Comment:
   The changes are rebased with the trunk as other PR got merged.  



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1121236675


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;
+                        try {
+                            index = new TransactionIndex(startOffset, file);
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        index.sanityCheck();
+                        return index;
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+    }
+
+    public void close() {
+        closed = true;
+        try {
+            cleanerThread.shutdown();
+        } catch (InterruptedException e) {
+            // ignore interrupted exception
+        }
+
+        // Close all the opened indexes.
+        synchronized (lock) {
+            entries.values().forEach(Entry::close);
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(entries);
+    }
+
+    public static class Entry {
+
+        public final OffsetIndex offsetIndex;
+        public final TimeIndex timeIndex;
+        public final TransactionIndex txnIndex;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        private boolean markedForCleanup = false;
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        public OffsetPosition lookupOffset(long targetOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+
+            try {
+                LogFileUtils.tryAll(Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                }));
+            } catch (Exception e) {
+                throw new KafkaException(e);
+            }
+        }
+
+        public void close() {
+            Arrays.asList(offsetIndex, timeIndex).forEach(index -> {
+                try {
+                    index.close();
+                } catch (Exception e) {
+                    // ignore exception.
+                }
+            });

Review Comment:
   Good point. We can use closeQuietly here, I do not remember why I avoided it for other indexes.



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


[GitHub] [kafka] satishd commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1449643362

   Thanks @showuon for your review. Addressed the comments with the latest commit. 


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


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1255762594


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";

Review Comment:
   nit
   
   could we group together public static together? Without that, visually it's a bit tricky to skim through the accessors while reading the code.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,582 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.debug("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");
+                        }
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String fileNameStr = fileNamePath.toString();
+                String name = fileNameStr.substring(0, fileNameStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                long offset = Long.parseLong(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            Uuid cacheKey = remoteLogSegmentMetadata.remoteLogSegmentId().id();
+            return internalCache.get(cacheKey, (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TransactionIndex index = new TransactionIndex(startOffset, file);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());

Review Comment:
   This is done as an optimization here for parallel processing. The `entry.close()` operation, performs close of 3 indexes per entry in the cache, hence, it closes 1024 * 3 = 3072 indexes. Closing of each index performs index resize and mmap unload which can be an expensive operation. Hence, we want to start the cleaner thread shutdown and instead of waiting for it to close, we want to continue perfomring other operations such as entry.close.
   
   That is why we do initiate and await separately.



##########
build.gradle:
##########
@@ -1704,6 +1704,7 @@ project(':storage') {
     implementation project(':storage:api')
     implementation project(':server-common')
     implementation project(':clients')
+    implementation libs.caffeine

Review Comment:
   should we remove it from `:core` since it is not being used there anymore?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,581 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use [[Caffeine]] cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");
+                        }
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String fileNameStr = fileNamePath.toString();
+                String name = fileNameStr.substring(0, fileNameStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            Uuid cacheKey = remoteLogSegmentMetadata.remoteLogSegmentId().id();
+            return internalCache.get(cacheKey, (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TransactionIndex index = new TransactionIndex(startOffset, file);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+

Review Comment:
   Yes, we don't need to explicitly need to set it to null. Beneath, it uses a hashmap-like data structure which would be GC'ed as usual. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1258418290


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,666 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index file {}", indexFile.getPath(), ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public static class Entry implements AutoCloseable {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;
+
+        // This lock is used to synchronize cleanup methods and read methods. This ensures that cleanup (which changes the
+        // underlying files of the index) isn't performed while a read is in-progress for the entry. This is required in
+        // addition to using the thread safe cache because, while the thread safety of the cache ensures that we can read
+        // entries concurrently, it does not ensure that we won't mutate underlying files belonging to an entry.
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        private boolean cleanStarted = false;
+
+        private boolean markedForCleanup = false;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        public OffsetIndex offsetIndex() {
+            return offsetIndex;
+        }
+
+        public TimeIndex timeIndex() {
+            return timeIndex;
+        }
+
+        public TransactionIndex txnIndex() {

Review Comment:
   This is for testing purposes, readlock will not help as the caller may mutate the returned index.  Updated the methods mentioning these are visible for testing.



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1259125780


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ * This class is thread safe.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    // Visible for testing
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);

Review Comment:
   Currently, it creates an empty file. There is already https://issues.apache.org/jira/browse/KAFKA-14993 to address that. I did not want to add those changes to this PR. @kamalcph was working on that cleanup. 



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


[GitHub] [kafka] satishd merged pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd merged PR #13275:
URL: https://github.com/apache/kafka/pull/13275


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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1123026606


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);

Review Comment:
   This exception is not directly thrown to the constructor. It is inside an anonymous inner class of `LinkedHashMap#removeEldestEntry`. This method is not declared with `IOException`. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1123045997


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.storage.internals.log;
+
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * 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.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    private final File cacheDir;
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+    private final Object lock = new Object();
+    private final RemoteStorageManager remoteStorageManager;
+    private final Map<Uuid, Entry> entries;
+    private final ShutdownableThread cleanerThread;
+
+    private volatile boolean closed = false;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        entries = new LinkedHashMap<Uuid, RemoteIndexCache.Entry>(maxSize, 0.75f, true) {
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<Uuid, RemoteIndexCache.Entry> eldest) {
+                if (this.size() > maxSize) {
+                    RemoteIndexCache.Entry entry = eldest.getValue();
+                    // Mark the entries for cleanup, background thread will clean them later.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    expiredIndexes.add(entry);
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!closed) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry $entry");
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        log.info("Cleaner thread was interrupted", ex);
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        if (cacheDir.mkdir())
+            log.info("Created Cache dir [{}] successfully", cacheDir);
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+
+                String pathStr = path.getFileName().toString();
+                String name = pathStr.substring(0, pathStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                if (!entries.containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    try {
+                        if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                            OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                            offsetIndex.sanityCheck();
+
+                            TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                            timeIndex.sanityCheck();
+
+                            TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                            txnIndex.sanityCheck();
+
+                            Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                            entries.put(uuid, entry);
+                        } else {
+                            // Delete all of them if any one of those indexes is not available for a specific segment id
+                            LogFileUtils.tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                        }
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (closed) throw new IllegalStateException("Instance is already closed.");
+
+        synchronized (lock) {
+            return entries.computeIfAbsent(remoteLogSegmentMetadata.remoteLogSegmentId().id(), (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        TransactionIndex index = null;
+                        try {
+                            index = new TransactionIndex(startOffset, file);
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        index.sanityCheck();
+                        return index;
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+    }
+
+    public void close() {
+        closed = true;
+        try {
+            cleanerThread.shutdown();
+        } catch (InterruptedException e) {
+            // ignore interrupted exception
+        }
+
+        // Close all the opened indexes.
+        synchronized (lock) {
+            entries.values().forEach(Entry::close);
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(entries);
+    }
+
+    public static class Entry {
+
+        public final OffsetIndex offsetIndex;
+        public final TimeIndex timeIndex;
+        public final TransactionIndex txnIndex;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        private boolean markedForCleanup = false;
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        public OffsetPosition lookupOffset(long targetOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));

Review Comment:
   Good catch! 



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


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257486930


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.

Review Comment:
   stale comment? (since it's private now)



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {

Review Comment:
   please doc visible for testing



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(internalCache.asMap());
+    }
+
+    public static class Entry {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;
+
+        // This lock is used to synchronize cleanup methods and read methods. This ensures that cleanup (which changes the
+        // underlying files of the index) isn't performed while a read is in-progress for the entry. This is required in
+        // addition to using the thread safe cache because, while the thread safety of the cache ensures that we can read
+        // entries concurrently, it does not ensure that we won't mutate underlying files belonging to an entry.
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        private boolean cleanStarted = false;
+
+        private boolean markedForCleanup = false;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        public OffsetIndex offsetIndex() {
+            return offsetIndex;
+        }
+
+        public TimeIndex timeIndex() {
+            return timeIndex;
+        }
+
+        public TransactionIndex txnIndex() {
+            return txnIndex;
+        }
+
+        // Visible for testing
+        public boolean isCleanStarted() {
+            return cleanStarted;
+        }
+
+        // Visible for testing
+        public boolean isMarkedForCleanup() {
+            return markedForCleanup;
+        }
+
+        public OffsetPosition lookupOffset(long targetOffset) {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    timeIndex.renameTo(new File(Utils.replaceSuffix(timeIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+            // no-op if clean is done already
+            if (!cleanStarted) {
+                cleanStarted = true;
+
+                List<StorageAction<Void, Exception>> actions = Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                });
+
+                tryAll(actions);
+            }
+        }
+
+        public void close() {
+            Utils.closeQuietly(offsetIndex, "OffsetIndex");

Review Comment:
   the write locks are missing for close() and cleanup(), is that intentional?



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);

Review Comment:
   please add the index path to the log
   ```
   at ${indexFile.toPath}", ex)
   ```



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);

Review Comment:
   Please add debug log similar to scala code. It helps determine what files were cleaned:
   
   ```
   if (Files.deleteIfExists(path))
             debug(s"Deleted file $path on cache initialization")
   ````



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {

Review Comment:
   visible for testing I assume? (please add a doc) Otherwise we should not be exposing internal cache entries directly.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {
+        return Collections.unmodifiableMap(internalCache.asMap());
+    }
+
+    public static class Entry {
+
+        private final OffsetIndex offsetIndex;
+        private final TimeIndex timeIndex;
+        private final TransactionIndex txnIndex;
+
+        // This lock is used to synchronize cleanup methods and read methods. This ensures that cleanup (which changes the
+        // underlying files of the index) isn't performed while a read is in-progress for the entry. This is required in
+        // addition to using the thread safe cache because, while the thread safety of the cache ensures that we can read
+        // entries concurrently, it does not ensure that we won't mutate underlying files belonging to an entry.
+        private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+        private boolean cleanStarted = false;
+
+        private boolean markedForCleanup = false;
+
+        public Entry(OffsetIndex offsetIndex, TimeIndex timeIndex, TransactionIndex txnIndex) {
+            this.offsetIndex = offsetIndex;
+            this.timeIndex = timeIndex;
+            this.txnIndex = txnIndex;
+        }
+
+        public OffsetIndex offsetIndex() {
+            return offsetIndex;
+        }
+
+        public TimeIndex timeIndex() {
+            return timeIndex;
+        }
+
+        public TransactionIndex txnIndex() {
+            return txnIndex;
+        }
+
+        // Visible for testing
+        public boolean isCleanStarted() {
+            return cleanStarted;
+        }
+
+        // Visible for testing
+        public boolean isMarkedForCleanup() {
+            return markedForCleanup;
+        }
+
+        public OffsetPosition lookupOffset(long targetOffset) {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+                else return offsetIndex.lookup(targetOffset);
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public OffsetPosition lookupTimestamp(long timestamp, long startingOffset) throws IOException {
+            lock.readLock().lock();
+            try {
+                if (markedForCleanup) throw new IllegalStateException("This entry is marked for cleanup");
+
+                TimestampOffset timestampOffset = timeIndex.lookup(timestamp);
+                return offsetIndex.lookup(Math.max(startingOffset, timestampOffset.offset));
+            } finally {
+                lock.readLock().unlock();
+            }
+        }
+
+        public void markForCleanup() throws IOException {
+            lock.writeLock().lock();
+            try {
+                if (!markedForCleanup) {
+                    markedForCleanup = true;
+                    offsetIndex.renameTo(new File(Utils.replaceSuffix(offsetIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    timeIndex.renameTo(new File(Utils.replaceSuffix(timeIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                    txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file().getPath(), "", LogFileUtils.DELETED_FILE_SUFFIX)));
+                }
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+
+        public void cleanup() throws IOException {
+            markForCleanup();
+            // no-op if clean is done already
+            if (!cleanStarted) {
+                cleanStarted = true;
+
+                List<StorageAction<Void, Exception>> actions = Arrays.asList(() -> {
+                    offsetIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    timeIndex.deleteIfExists();
+                    return null;
+                }, () -> {
+                    txnIndex.deleteIfExists();
+                    return null;
+                });
+
+                tryAll(actions);
+            }
+        }
+
+        public void close() {
+            Utils.closeQuietly(offsetIndex, "OffsetIndex");
+            Utils.closeQuietly(timeIndex, "TimeIndex");
+            Utils.closeQuietly(txnIndex, "TransactionIndex");
+        }
+
+        @Override
+        public String toString() {

Review Comment:
   This just prints the object reference, that is why the scala file uses the file name instead.



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());

Review Comment:
   I was about to create a ticket for this but while we are lurking in this code base, might as well change it right now. We need to add REPLACE_EXISTING here. Otherwise, we are not overwriting the corrupted file. We need a to add a test that fails for this use case as well. We can do that in a ticket (or in this PR is you prefer that).



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257700736


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            // while this thread was waiting for lock, another thread may have changed the value of isRemoteIndexCacheClosed.
+            // check for index close again
+            if (isRemoteIndexCacheClosed.get()) {
+                throw new IllegalStateException("Unable to fetch index for segment id="
+                        + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Index instance is already closed.");
+            }
+
+            return internalCache.get(remoteLogSegmentMetadata.remoteLogSegmentId().id(),
+                    (Uuid uuid) -> createCacheEntry(remoteLogSegmentMetadata));
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    private RemoteIndexCache.Entry createCacheEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        long startOffset = remoteLogSegmentMetadata.startOffset();
+
+        try {
+            File offsetIndexFile = remoteOffsetIndexFile(cacheDir, remoteLogSegmentMetadata);
+            OffsetIndex offsetIndex = loadIndexFile(offsetIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File timeIndexFile = remoteTimeIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TimeIndex timeIndex = loadIndexFile(timeIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+            File txnIndexFile = remoteTransactionIndexFile(cacheDir, remoteLogSegmentMetadata);
+            TransactionIndex txnIndex = loadIndexFile(txnIndexFile, remoteLogSegmentMetadata, rlsMetadata -> {
+                try {
+                    return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                } catch (RemoteStorageException e) {
+                    throw new KafkaException(e);
+                }
+            }, file -> {
+                try {
+                    TransactionIndex index = new TransactionIndex(startOffset, file);
+                    index.sanityCheck();
+                    return index;
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+
+            return new Entry(offsetIndex, timeIndex, txnIndex);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+
+                log.info("Close completed for RemoteIndexCache");
+            } catch (InterruptedException e) {
+                throw new KafkaException(e);
+            } finally {
+                lock.writeLock().unlock();
+            }
+        }
+    }
+
+    public Map<Uuid, Entry> entries() {

Review Comment:
   It was meant for testing but it returned unmodifiable collection so that callers can not modify it. Anyways, this method is no more used with the latest changes. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257708611


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());

Review Comment:
   I made the change in this PR, we can have the respective test in a followup PR.



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


[GitHub] [kafka] satishd commented on pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13275:
URL: https://github.com/apache/kafka/pull/13275#issuecomment-1628253996

   Thanks @junrao for the review comment. Addressed them with the latest commits and/or inline.


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


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257912685


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,657 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);
+
+                    // Create entries for each path if all the index files exist.
+                    if (Files.exists(offsetIndexFile.toPath()) &&
+                            Files.exists(timestampIndexFile.toPath()) &&
+                            Files.exists(txnIndexFile.toPath())) {
+                        long offset = offsetFromRemoteIndexFileName(indexFileName);
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+        log.info("RemoteIndexCache starts up in {} ms.", Time.SYSTEM.hiResClockMs() - start);
+    }
+
+    private <T> T loadIndexFile(File file, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, file.getName());
+        T index = null;
+        if (Files.exists(indexFile.toPath())) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());

Review Comment:
   sounds good.



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


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1257913979


##########
build.gradle:
##########
@@ -1704,6 +1704,7 @@ project(':storage') {
     implementation project(':storage:api')
     implementation project(':server-common')
     implementation project(':clients')
+    implementation libs.caffeine

Review Comment:
   thank you. This is better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1254737901


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,581 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use [[Caffeine]] cache instead of implementing a thread safe LRU cache on our own.
+     *
+     * Visible for testing.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    try {
+                        entry.markForCleanup();
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                    if (!expiredIndexes.offer(entry)) {
+                        log.error("Error while inserting entry {} into the cleaner queue", entry);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public LinkedBlockingQueue<Entry> expiredIndexes() {
+        return expiredIndexes;
+    }
+
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                while (!isRemoteIndexCacheClosed.get()) {
+                    try {
+                        Entry entry = expiredIndexes.take();
+                        log.info("Cleaning up index entry {}", entry);
+                        entry.cleanup();
+                    } catch (InterruptedException ex) {
+                        // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                        if (!isRemoteIndexCacheClosed.get()) {
+                            log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
+                            throw new KafkaException(ex);
+                        } else {
+                            log.debug("Cleaner thread was interrupted on cache shutdown");
+                        }
+                    } catch (Exception ex) {
+                        log.error("Error occurred while fetching/cleaning up expired entry", ex);
+                    }
+                }
+            }
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
+                    try {
+                        Files.deleteIfExists(path);
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String fileNameStr = fileNamePath.toString();
+                String name = fileNameStr.substring(0, fileNameStr.lastIndexOf("_") + 1);
+
+                // Create entries for each path if all the index files exist.
+                int firstIndex = name.indexOf('_');
+                int offset = Integer.parseInt(name.substring(0, firstIndex));
+                Uuid uuid = Uuid.fromString(name.substring(firstIndex + 1, name.lastIndexOf('_')));
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    File offsetIndexFile = new File(cacheDir, name + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, name + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, name + TXN_INDEX_FILE_SUFFIX);
+
+                    if (offsetIndexFile.exists() && timestampIndexFile.exists() && txnIndexFile.exists()) {
+
+                        OffsetIndex offsetIndex = new OffsetIndex(offsetIndexFile, offset, Integer.MAX_VALUE, false);
+                        offsetIndex.sanityCheck();
+
+                        TimeIndex timeIndex = new TimeIndex(timestampIndexFile, offset, Integer.MAX_VALUE, false);
+                        timeIndex.sanityCheck();
+
+                        TransactionIndex txnIndex = new TransactionIndex(offset, txnIndexFile);
+                        txnIndex.sanityCheck();
+
+                        Entry entry = new Entry(offsetIndex, timeIndex, txnIndex);
+                        internalCache.put(uuid, entry);
+                    } else {
+                        // Delete all of them if any one of those indexes is not available for a specific segment id
+                        tryAll(Arrays.asList(
+                                () -> {
+                                    Files.deleteIfExists(offsetIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(timestampIndexFile.toPath());
+                                    return null;
+                                },
+                                () -> {
+                                    Files.deleteIfExists(txnIndexFile.toPath());
+                                    return null;
+                                }));
+                    }
+                }
+            }
+        }
+    }
+
+    private <T> T loadIndexFile(String fileName, String suffix, RemoteLogSegmentMetadata remoteLogSegmentMetadata,
+                                Function<RemoteLogSegmentMetadata, InputStream> fetchRemoteIndex,
+                                Function<File, T> readIndex) throws IOException {
+        File indexFile = new File(cacheDir, fileName + suffix);
+        T index = null;
+        if (indexFile.exists()) {
+            try {
+                index = readIndex.apply(indexFile);
+            } catch (CorruptRecordException ex) {
+                log.info("Error occurred while loading the stored index", ex);
+            }
+        }
+
+        if (index == null) {
+            File tmpIndexFile = new File(indexFile.getParentFile(), indexFile.getName() + RemoteIndexCache.TMP_FILE_SUFFIX);
+
+            try (InputStream inputStream = fetchRemoteIndex.apply(remoteLogSegmentMetadata);) {
+                Files.copy(inputStream, tmpIndexFile.toPath());
+            }
+
+            Utils.atomicMoveWithFallback(tmpIndexFile.toPath(), indexFile.toPath(), false);
+            index = readIndex.apply(indexFile);
+        }
+
+        return index;
+    }
+
+    public Entry getIndexEntry(RemoteLogSegmentMetadata remoteLogSegmentMetadata) {
+        if (isRemoteIndexCacheClosed.get()) throw new IllegalStateException("Unable to fetch index for " +
+                "segment id=" + remoteLogSegmentMetadata.remoteLogSegmentId().id() + ". Instance is already closed.");
+
+        lock.readLock().lock();
+        try {
+            Uuid cacheKey = remoteLogSegmentMetadata.remoteLogSegmentId().id();
+            return internalCache.get(cacheKey, (Uuid uuid) -> {
+                long startOffset = remoteLogSegmentMetadata.startOffset();
+                // uuid.toString uses URL encoding which is safe for filenames and URLs.
+                String fileName = startOffset + "_" + uuid.toString() + "_";
+
+                try {
+                    OffsetIndex offsetIndex = loadIndexFile(fileName, INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.OFFSET);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            OffsetIndex index = new OffsetIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TimeIndex timeIndex = loadIndexFile(fileName, TIME_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TIMESTAMP);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TimeIndex index = new TimeIndex(file, startOffset, Integer.MAX_VALUE, false);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    TransactionIndex txnIndex = loadIndexFile(fileName, TXN_INDEX_FILE_SUFFIX, remoteLogSegmentMetadata, rlsMetadata -> {
+                        try {
+                            return remoteStorageManager.fetchIndex(rlsMetadata, IndexType.TRANSACTION);
+                        } catch (RemoteStorageException e) {
+                            throw new KafkaException(e);
+                        }
+                    }, file -> {
+                        try {
+                            TransactionIndex index = new TransactionIndex(startOffset, file);
+                            index.sanityCheck();
+                            return index;
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                    });
+
+                    return new Entry(offsetIndex, timeIndex, txnIndex);
+                } catch (IOException e) {
+                    throw new KafkaException(e);
+                }
+            });
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupOffset(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long offset) {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupOffset(offset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public int lookupTimestamp(RemoteLogSegmentMetadata remoteLogSegmentMetadata, long timestamp, long startingOffset) throws IOException {
+        lock.readLock().lock();
+        try {
+            return getIndexEntry(remoteLogSegmentMetadata).lookupTimestamp(timestamp, startingOffset).position;
+        } finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    public void close() {
+        // Make close idempotent and ensure no more reads allowed from henceforth. The in-progress reads will continue to
+        // completion (release the read lock) and then close will begin executing. Cleaner thread will immediately stop work.
+        if (!isRemoteIndexCacheClosed.getAndSet(true)) {
+            lock.writeLock().lock();
+            try {
+                log.info("Close initiated for RemoteIndexCache. Cache stats={}. Cache entries pending delete={}",
+                        internalCache.stats(), expiredIndexes.size());
+                // Initiate shutdown for cleaning thread
+                boolean shutdownRequired = cleanerThread.initiateShutdown();
+                // Close all the opened indexes to force unload mmap memory. This does not delete the index files from disk.
+                internalCache.asMap().forEach((uuid, entry) -> entry.close());
+                // wait for cleaner thread to shutdown
+                if (shutdownRequired) cleanerThread.awaitShutdown();
+
+                // Note that internal cache does not require explicit cleaning/closing. We don't want to invalidate or cleanup
+                // the cache as both would lead to triggering of removal listener.
+

Review Comment:
   I did not want to make this non final and set it null. That is not really needed as this method is called while the Kafka broker shutsdown and this will be GCed anyways. 



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


[GitHub] [kafka] satishd commented on a diff in pull request #13275: KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13275:
URL: https://github.com/apache/kafka/pull/13275#discussion_r1259125780


##########
storage/src/main/java/org/apache/kafka/storage/internals/log/RemoteIndexCache.java:
##########
@@ -0,0 +1,671 @@
+/*
+ * 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.storage.internals.log;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageException;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
+import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.storage.internals.log.LogFileUtils.INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TIME_INDEX_FILE_SUFFIX;
+import static org.apache.kafka.storage.internals.log.LogFileUtils.TXN_INDEX_FILE_SUFFIX;
+
+/**
+ * This is a LFU (Least Frequently Used) 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. The cache is re-initialized from the index files on disk on startup, if the index files are available.
+ *
+ * The cache contains a garbage collection thread which will delete the files for entries that have been removed from
+ * the cache.
+ *
+ * Note that closing this cache does not delete the index files on disk.
+ * Note that the cache eviction policy is based on the default implementation of Caffeine i.e.
+ * <a href="https://github.com/ben-manes/caffeine/wiki/Efficiency">Window TinyLfu</a>. TinyLfu relies on a frequency
+ * sketch to probabilistically estimate the historic usage of an entry.
+ *
+ * This class is thread safe.
+ */
+public class RemoteIndexCache implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(RemoteIndexCache.class);
+    private static final String TMP_FILE_SUFFIX = ".tmp";
+
+    public static final String REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD = "remote-log-index-cleaner";
+    public static final String DIR_NAME = "remote-log-index-cache";
+
+    /**
+     * Directory where the index files will be stored on disk.
+     */
+    private final File cacheDir;
+
+    /**
+     * Represents if the cache is closed or not. Closing the cache is an irreversible operation.
+     */
+    private final AtomicBoolean isRemoteIndexCacheClosed = new AtomicBoolean(false);
+
+    /**
+     * Unbounded queue containing the removed entries from the cache which are waiting to be garbage collected.
+     */
+    private final LinkedBlockingQueue<Entry> expiredIndexes = new LinkedBlockingQueue<>();
+
+    /**
+     * Lock used to synchronize close with other read operations. This ensures that when we close, we don't have any other
+     * concurrent reads in-progress.
+     */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /**
+     * Actual cache implementation that this file wraps around.
+     *
+     * The requirements for this internal cache is as follows:
+     * 1. Multiple threads should be able to read concurrently.
+     * 2. Fetch for missing keys should not block read for available keys.
+     * 3. Only one thread should fetch for a specific key.
+     * 4. Should support LRU-like policy.
+     *
+     * We use {@link Caffeine} cache instead of implementing a thread safe LRU cache on our own.
+     */
+    private final Cache<Uuid, Entry> internalCache;
+    private final RemoteStorageManager remoteStorageManager;
+    private final ShutdownableThread cleanerThread;
+
+    public RemoteIndexCache(RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this(1024, remoteStorageManager, logDir);
+    }
+
+    /**
+     * Creates RemoteIndexCache with the given configs.
+     *
+     * @param maxSize              maximum number of segment index entries to be cached.
+     * @param remoteStorageManager RemoteStorageManager instance, to be used in fetching indexes.
+     * @param logDir               log directory
+     */
+    public RemoteIndexCache(int maxSize, RemoteStorageManager remoteStorageManager, String logDir) throws IOException {
+        this.remoteStorageManager = remoteStorageManager;
+        cacheDir = new File(logDir, DIR_NAME);
+
+        internalCache = Caffeine.newBuilder()
+                .maximumSize(maxSize)
+                // removeListener is invoked when either the entry is invalidated (means manual removal by the caller) or
+                // evicted (means removal due to the policy)
+                .removalListener((Uuid key, Entry entry, RemovalCause cause) -> {
+                    // Mark the entries for cleanup and add them to the queue to be garbage collected later by the background thread.
+                    if (entry != null) {
+                        try {
+                            entry.markForCleanup();
+                        } catch (IOException e) {
+                            throw new KafkaException(e);
+                        }
+                        if (!expiredIndexes.offer(entry)) {
+                            log.error("Error while inserting entry {} for key {} into the cleaner queue because queue is full.", entry, key);
+                        }
+                    } else {
+                        log.error("Received entry as null for key {} when the it is removed from the cache.", key);
+                    }
+                }).build();
+
+        init();
+
+        // Start cleaner thread that will clean the expired entries.
+        cleanerThread = createCleanerThread();
+        cleanerThread.start();
+    }
+
+    public Collection<Entry> expiredIndexes() {
+        return Collections.unmodifiableCollection(expiredIndexes);
+    }
+
+    // Visible for testing
+    public Cache<Uuid, Entry> internalCache() {
+        return internalCache;
+    }
+
+    // Visible for testing
+    public ShutdownableThread cleanerThread() {
+        return cleanerThread;
+    }
+
+    private ShutdownableThread createCleanerThread() {
+        ShutdownableThread thread = new ShutdownableThread("remote-log-index-cleaner") {
+            public void doWork() {
+                try {
+                    Entry entry = expiredIndexes.take();
+                    log.debug("Cleaning up index entry {}", entry);
+                    entry.cleanup();
+                } catch (InterruptedException ie) {
+                    // cleaner thread should only be interrupted when cache is being closed, else it's an error
+                    if (!isRemoteIndexCacheClosed.get()) {
+                        log.error("Cleaner thread received interruption but remote index cache is not closed", ie);
+                        // propagate the InterruptedException outside to correctly close the thread.
+                        throw new KafkaException(ie);
+                    } else {
+                        log.debug("Cleaner thread was interrupted on cache shutdown");
+                    }
+                } catch (Exception ex) {
+                    // do not exit for exceptions other than InterruptedException
+                    log.error("Error occurred while cleaning up expired entry", ex);
+                }
+            }
+
+        };
+        thread.setDaemon(true);
+
+        return thread;
+    }
+
+    private void init() throws IOException {
+        long start = Time.SYSTEM.hiResClockMs();
+
+        try {
+            Files.createDirectory(cacheDir.toPath());
+            log.info("Created new file {} for RemoteIndexCache", cacheDir);
+        } catch (FileAlreadyExistsException e) {
+            log.info("RemoteIndexCache directory {} already exists. Re-using the same directory.", cacheDir);
+        } catch (Exception e) {
+            log.error("Unable to create directory {} for RemoteIndexCache.", cacheDir, e);
+            throw new KafkaException(e);
+        }
+
+        // Delete any .deleted or .tmp files remained from the earlier run of the broker.
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            paths.forEach(path -> {
+                if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX) ||
+                        path.endsWith(TMP_FILE_SUFFIX)) {
+                    try {
+                        if (Files.deleteIfExists(path)) {
+                            log.debug("Deleted file path {} on cache initialization", path);
+                        }
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+            });
+        }
+
+        try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
+            Iterator<Path> iterator = paths.iterator();
+            while (iterator.hasNext()) {
+                Path path = iterator.next();
+                Path fileNamePath = path.getFileName();
+                if (fileNamePath == null)
+                    throw new KafkaException("Empty file name in remote index cache directory: " + cacheDir);
+
+                String indexFileName = fileNamePath.toString();
+                Uuid uuid = remoteLogSegmentIdFromRemoteIndexFileName(indexFileName);
+
+                // It is safe to update the internalCache non-atomically here since this function is always called by a single
+                // thread only.
+                if (!internalCache.asMap().containsKey(uuid)) {
+                    String fileNameWithoutSuffix = indexFileName.substring(0, indexFileName.indexOf("."));
+                    File offsetIndexFile = new File(cacheDir, fileNameWithoutSuffix + INDEX_FILE_SUFFIX);
+                    File timestampIndexFile = new File(cacheDir, fileNameWithoutSuffix + TIME_INDEX_FILE_SUFFIX);
+                    File txnIndexFile = new File(cacheDir, fileNameWithoutSuffix + TXN_INDEX_FILE_SUFFIX);

Review Comment:
   Currently, it creates an empty file. There is already https://issues.apache.org/jira/browse/KAFKA-14993 to address that. I did not want to add those changes to this PR. @kamalcph was working on that cleanup. 



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