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

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

Reply via email to