divijvaidya commented on code in PR #13850:
URL: https://github.com/apache/kafka/pull/13850#discussion_r1231951562


##########
core/src/main/scala/kafka/log/remote/RemoteIndexCache.scala:
##########
@@ -37,88 +40,125 @@ object RemoteIndexCache {
   val TmpFileSuffix = ".tmp"
 }
 
-class Entry(val offsetIndex: OffsetIndex, val timeIndex: TimeIndex, val 
txnIndex: TransactionIndex) {
+class Entry(val offsetIndex: OffsetIndex, val timeIndex: TimeIndex, val 
txnIndex: TransactionIndex) extends AutoCloseable {
   private var markedForCleanup: Boolean = false
-  private val lock: ReentrantReadWriteLock = new ReentrantReadWriteLock()
+  private val entryLock: ReentrantReadWriteLock = new ReentrantReadWriteLock()
 
   def lookupOffset(targetOffset: Long): OffsetPosition = {
-    CoreUtils.inLock(lock.readLock()) {
+    inReadLock(entryLock) {
       if (markedForCleanup) throw new IllegalStateException("This entry is 
marked for cleanup")
       else offsetIndex.lookup(targetOffset)
     }
   }
 
   def lookupTimestamp(timestamp: Long, startingOffset: Long): OffsetPosition = 
{
-    CoreUtils.inLock(lock.readLock()) {
+    inReadLock(entryLock) {
       if (markedForCleanup) throw new IllegalStateException("This entry is 
marked for cleanup")
-
       val timestampOffset = timeIndex.lookup(timestamp)
       offsetIndex.lookup(math.max(startingOffset, timestampOffset.offset))
     }
   }
 
   def markForCleanup(): Unit = {
-    CoreUtils.inLock(lock.writeLock()) {
+    inWriteLock(entryLock) {
       if (!markedForCleanup) {
         markedForCleanup = true
         Array(offsetIndex, timeIndex).foreach(index =>
           index.renameTo(new File(Utils.replaceSuffix(index.file.getPath, "", 
LogFileUtils.DELETED_FILE_SUFFIX))))
+        // txn index needs to be renamed separately since it's not of type 
AbstractIndex
         txnIndex.renameTo(new File(Utils.replaceSuffix(txnIndex.file.getPath, 
"",
           LogFileUtils.DELETED_FILE_SUFFIX)))
       }
     }
   }
 
+  /**
+   * Deletes the index files from the disk. Invoking #close is not required 
prior to this function.
+   */
   def cleanup(): Unit = {
     markForCleanup()
     CoreUtils.tryAll(Seq(() => offsetIndex.deleteIfExists(), () => 
timeIndex.deleteIfExists(), () => txnIndex.deleteIfExists()))
   }
 
+  /**
+   * Calls the underlying close method for each index which may lead to 
releasing resources such as mmap.
+   * This function does not delete the index files.
+   */
   def close(): Unit = {
-    Array(offsetIndex, timeIndex).foreach(index => try {
-      index.close()
-    } catch {
-      case _: Exception => // ignore error.
-    })
+    Utils.closeQuietly(offsetIndex, "Closing the offset index.")
+    Utils.closeQuietly(timeIndex, "Closing the time index.")
     Utils.closeQuietly(txnIndex, "Closing the transaction index.")
   }
 }
 
 /**
  * This is a LRU cache of remote index files stored in 
`$logdir/remote-log-index-cache`. This is helpful to avoid
- * re-fetching the index files like offset, time indexes from the remote 
storage for every fetch call.
+ * 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 this cache is not strictly based on a LRU policy. It 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.
  *
  * @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
  */
+@threadsafe
 class RemoteIndexCache(maxSize: Int = 1024, remoteStorageManager: 
RemoteStorageManager, logDir: String)
-  extends Logging with Closeable {
-
-  val cacheDir = new File(logDir, DirName)
-  @volatile var closed = false
-
-  val expiredIndexes = new LinkedBlockingQueue[Entry]()
-  val lock = new Object()
-
-  val entries: util.Map[Uuid, Entry] = new java.util.LinkedHashMap[Uuid, 
Entry](maxSize / 2,
-    0.75f, true) {
-    override def removeEldestEntry(eldest: util.Map.Entry[Uuid, Entry]): 
Boolean = {
-      if (this.size() > maxSize) {
-        val entry = eldest.getValue
-        // Mark the entries for cleanup, background thread will clean them 
later.
-        entry.markForCleanup()
-        expiredIndexes.add(entry)
-        true
-      } else {
-        false
-      }
-    }
-  }
+  extends Logging with AutoCloseable {
+  /**
+   * Directory where the index files will be stored on disk.
+   */
+  private val cacheDir = new File(logDir, DirName)
+  /**
+   * Represents if the cache is closed or not. Closing the cache is an 
irreversible operation.
+   */
+  private val closed: AtomicBoolean = new AtomicBoolean(false)
+  /**
+   * Unbounded queue containing the removed entries from the cache which are 
waiting to be garbage collected.
+   */
+  private val expiredIndexes = new LinkedBlockingQueue[Entry]()
+  /**
+   * 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 policy.
+   *
+   * We use [[Caffeine]] cache instead of implementing a thread safe LRU cache 
on our own.
+   *
+   * Visible for testing.
+   */
+  private[remote] var internalCache: Cache[Uuid, Entry] = 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, entry: Entry, _: RemovalCause) => {
+      // Mark the entries for cleanup and add them to the queue to be garbage 
collected later by the background thread.
+      entry.markForCleanup()
+      expiredIndexes.add(entry)
+    })
+    .build[Uuid, Entry]()
+
 
   private def init(): Unit = {
-    if (cacheDir.mkdir())
+    try {
+      Files.createDirectory(cacheDir.toPath)
       info(s"Created $cacheDir successfully")
+    } catch {
+      case _: FileAlreadyExistsException =>
+        info(s"RemoteIndexCache directory $cacheDir already exists. Re-using 
the same directory.")
+      case e: Exception =>
+        error(s"Unable to create directory $cacheDir for RemoteIndexCache.", e)
+        throw new IllegalArgumentException(e)

Review Comment:
   I just checked the server initialization code and seems like we just 
propagate the underlying exception outside without wrapping in KafkaException 
(probably because this exception doesn't go to clients). I will do the same.



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