showuon commented on code in PR #13850: URL: https://github.com/apache/kafka/pull/13850#discussion_r1233805956
########## core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala: ########## @@ -170,44 +190,173 @@ class RemoteIndexCacheTest { assertThrows(classOf[IllegalStateException], () => cache.getIndexEntry(metadataList.head)) } + @Test + def testCloseIsIdempotent(): Unit = { + val spyCleanerThread = spy(cache.cleanerThread) + cache.cleanerThread = spyCleanerThread + cache.close() + cache.close() + // verify that cleanup is only called once + verify(spyCleanerThread).initiateShutdown() + } + + @Test + def testClose(): Unit = { + val spyInternalCache = spy(cache.internalCache) + val spyCleanerThread = spy(cache.cleanerThread) + + // replace with new spy cache + cache.internalCache = spyInternalCache + cache.cleanerThread = spyCleanerThread + + // use the cache + val tpId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)) + val metadataList = generateRemoteLogSegmentMetadata(size = 1, tpId) + val entry = cache.getIndexEntry(metadataList.head) + + val spyTxnIndex = spy(entry.txnIndex) + val spyOffsetIndex = spy(entry.offsetIndex) + val spyTimeIndex = spy(entry.timeIndex) + // remove this entry and replace with spied entry + cache.internalCache.invalidateAll() + cache.internalCache.put(metadataList.head.remoteLogSegmentId().id(), new Entry(spyOffsetIndex, spyTimeIndex, spyTxnIndex)) + + // close the cache + cache.close() + + // cleaner thread should be closed properly + verify(spyCleanerThread).initiateShutdown() + verify(spyCleanerThread).awaitShutdown() + + // close for all index entries must be invoked + verify(spyTxnIndex).close() + verify(spyOffsetIndex).close() + verify(spyTimeIndex).close() + + // index files must not be deleted + verify(spyTxnIndex, times(0)).deleteIfExists() + verify(spyOffsetIndex, times(0)).deleteIfExists() + verify(spyTimeIndex, times(0)).deleteIfExists() + } + + @Test + def testConcurrentReadWriteAccessForCache(): Unit = { + val tpId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)) + val metadataList = generateRemoteLogSegmentMetadata(size = 3, tpId) + + assertCacheSize(0) + // getIndex for first time will call rsm#fetchIndex + cache.getIndexEntry(metadataList.head) + assertCacheSize(1) + verifyFetchIndexInvocation(count = 1, Seq(IndexType.OFFSET, IndexType.TIMESTAMP)) + reset(rsm) + + // Simulate a concurrency situation where one thread is reading the entry already present in the cache (cache hit) + // and the other thread is reading an entry which is not available in the cache (cache miss). The expected behaviour + // is for the former thread to succeed while latter is fetching from rsm. + // In this this test we simulate the situation using latches. We perform the following operations: + // 1. Start the CacheMiss thread and wait until it starts executing the rsm.fetchIndex + // 2. Block the CacheMiss thread inside the call to rsm.fetchIndex. + // 3. Start the CacheHit thread. Assert that it performs a successful read. + // 4. On completion of successful read by CacheHit thread, signal the CacheMiss thread to release it's block. + // 5. Validate that the test passes. If the CacheMiss thread was blocking the CacheHit thread, the test will fail. + // + val latchForCacheHit = new CountDownLatch(1) + val latchForCacheMiss = new CountDownLatch(1) + + val readerCacheHit = (() => { + // Wait for signal to start executing the read + logger.debug(s"Waiting for signal to begin read from ${Thread.currentThread()}") + latchForCacheHit.await() + val entry = cache.getIndexEntry(metadataList.head) + assertNotNull(entry) + // Signal the CacheMiss to unblock itself + logger.debug(s"Signaling CacheMiss to unblock from ${Thread.currentThread()}") + latchForCacheMiss.countDown() + }): Runnable + + when(rsm.fetchIndex(any(classOf[RemoteLogSegmentMetadata]), any(classOf[IndexType]))) + .thenAnswer(_ => { + logger.debug(s"Signaling CacheHit to begin read from ${Thread.currentThread()}") + latchForCacheHit.countDown() + logger.debug("Waiting for signal to complete rsm fetch from" + Thread.currentThread()) Review Comment: nit: use `s"..."` ########## core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala: ########## @@ -117,51 +128,60 @@ class RemoteIndexCacheTest { @Test def testCacheEntryExpiry(): Unit = { - val cache = new RemoteIndexCache(maxSize = 2, rsm, logDir = logDir.toString) + // close existing cache created in test setup before creating a new one + Utils.closeQuietly(cache, "RemoteIndexCache created for unit test") Review Comment: If we don't do this at the beginning of the test, will it fail the tests? I thought we've already done it at the `AfterEach` ########## core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala: ########## @@ -170,44 +190,173 @@ class RemoteIndexCacheTest { assertThrows(classOf[IllegalStateException], () => cache.getIndexEntry(metadataList.head)) } + @Test + def testCloseIsIdempotent(): Unit = { + val spyCleanerThread = spy(cache.cleanerThread) + cache.cleanerThread = spyCleanerThread + cache.close() + cache.close() + // verify that cleanup is only called once + verify(spyCleanerThread).initiateShutdown() + } + + @Test + def testClose(): Unit = { + val spyInternalCache = spy(cache.internalCache) + val spyCleanerThread = spy(cache.cleanerThread) + + // replace with new spy cache + cache.internalCache = spyInternalCache + cache.cleanerThread = spyCleanerThread + + // use the cache + val tpId = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)) + val metadataList = generateRemoteLogSegmentMetadata(size = 1, tpId) + val entry = cache.getIndexEntry(metadataList.head) + + val spyTxnIndex = spy(entry.txnIndex) + val spyOffsetIndex = spy(entry.offsetIndex) + val spyTimeIndex = spy(entry.timeIndex) + // remove this entry and replace with spied entry + cache.internalCache.invalidateAll() + cache.internalCache.put(metadataList.head.remoteLogSegmentId().id(), new Entry(spyOffsetIndex, spyTimeIndex, spyTxnIndex)) + + // close the cache + cache.close() + + // cleaner thread should be closed properly + verify(spyCleanerThread).initiateShutdown() + verify(spyCleanerThread).awaitShutdown() + + // close for all index entries must be invoked + verify(spyTxnIndex).close() + verify(spyOffsetIndex).close() + verify(spyTimeIndex).close() + + // index files must not be deleted + verify(spyTxnIndex, times(0)).deleteIfExists() + verify(spyOffsetIndex, times(0)).deleteIfExists() + verify(spyTimeIndex, times(0)).deleteIfExists() + } + + @Test + def testConcurrentReadWriteAccessForCache(): Unit = { Review Comment: nice test! ########## 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 Review Comment: Since we're not a completely LRU cache anymore, should we update this line to avoid confusion? -- 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