@ -23,7 +23,7 @@ import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
@@ -23,7 +23,7 @@ import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
import org.apache.kafka.server.log.remote.storage. { RemoteLogSegmentId , RemoteLogSegmentMetadata , RemoteResourceNotFoundException , RemoteStorageManager }
import org.apache.kafka.server.util.MockTime
import org.apache.kafka.storage.internals.log.RemoteIndexCache. { Entry , REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD , remoteDeletedSuffixIndexFileName , remoteOffsetIndexFile , remoteOffsetIndexFileName , remoteTimeIndexFile , remoteTimeIndexFileName , remoteTransactionIndexFile , remoteTransactionIndexFileName }
import org.apache.kafka.storage.internals.log.RemoteIndexCache. { DIR_NAME , Entry , REMOTE_LOG_INDEX_CACHE_CLEANER_THREAD , remoteDeletedSuffixIndexFileName , remoteOffsetIndexFile , remoteOffsetIndexFileName , remoteTimeIndexFile , remoteTimeIndexFileName , remoteTransactionIndexFile , remoteTransactionIndexFileName }
import org.apache.kafka.storage.internals.log. { AbortedTxn , CorruptIndexException , LogFileUtils , OffsetIndex , OffsetPosition , RemoteIndexCache , TimeIndex , TransactionIndex }
import org.apache.kafka.test. { TestUtils => JTestUtils }
import org.junit.jupiter.api.Assertions._
@ -32,6 +32,7 @@ import org.junit.jupiter.params.ParameterizedTest
@@ -32,6 +32,7 @@ import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.EnumSource
import org.mockito.ArgumentMatchers
import org.mockito.ArgumentMatchers.any
import org.mockito.invocation.InvocationOnMock
import org.mockito.Mockito._
import org.slf4j. { Logger , LoggerFactory }
@ -39,7 +40,7 @@ import java.io.{File, FileInputStream, IOException, PrintWriter}
@@ -39,7 +40,7 @@ import java.io.{File, FileInputStream, IOException, PrintWriter}
import java.nio.file. { Files , NoSuchFileException , Paths }
import java.util
import java.util. { Collections , Optional }
import java.util.concurrent. { CountDownLatch , Executors , TimeUnit }
import java.util.concurrent. { CountDownLatch , Executors , Future , TimeUnit }
import scala.collection.mutable
class RemoteIndexCacheTest {
@ -138,8 +139,8 @@ class RemoteIndexCacheTest {
@@ -138,8 +139,8 @@ class RemoteIndexCacheTest {
. thenAnswer ( ans => {
val metadata = ans . getArgument [ RemoteLogSegmentMetadata ] ( 0 )
val indexType = ans . getArgument [ IndexType ] ( 1 )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata , tpDir )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata , tpDir )
maybeAppendIndexEntries ( offsetIdx , timeIdx )
indexType match {
case IndexType . OFFSET => new FileInputStream ( offsetIdx . file )
@ -249,7 +250,7 @@ class RemoteIndexCacheTest {
@@ -249,7 +250,7 @@ class RemoteIndexCacheTest {
}
@Test
def testCacheEntryIsDeletedOnInvalidation ( ) : Unit = {
def testCacheEntryIsDeletedOnRemoval ( ) : Unit = {
def getIndexFileFromDisk ( suffix : String ) = {
Files . walk ( tpDir . toPath )
. filter ( Files . isRegularFile ( _ ) )
@ -271,8 +272,8 @@ class RemoteIndexCacheTest {
@@ -271,8 +272,8 @@ class RemoteIndexCacheTest {
// no expired entries yet
assertEquals ( 0 , cache . expiredIndexes . size , "expiredIndex queue should be zero at start of test" )
// invalidate the cache . it should async mark the entry for removal
cache . internalCache . invalidat e( internalIndexKey )
// call remove function to mark the entry for removal
cache . remov e( internalIndexKey )
// wait until entry is marked for deletion
TestUtils . waitUntilTrue ( ( ) => cacheEntry . isMarkedForCleanup ,
@ -672,6 +673,84 @@ class RemoteIndexCacheTest {
@@ -672,6 +673,84 @@ class RemoteIndexCacheTest {
verifyFetchIndexInvocation ( count = 1 )
}
@Test
def testConcurrentRemoveReadForCache ( ) : Unit = {
// Create a spy Cache Entry
val rlsMetadata = new RemoteLogSegmentMetadata ( RemoteLogSegmentId . generateNew ( idPartition ) , baseOffset , lastOffset ,
time . milliseconds ( ) , brokerId , time . milliseconds ( ) , segmentSize , Collections . singletonMap ( 0 , 0L ) )
val timeIndex = spy ( createTimeIndexForSegmentMetadata ( rlsMetadata , new File ( tpDir , DIR_NAME ) ) )
val txIndex = spy ( createTxIndexForSegmentMetadata ( rlsMetadata , new File ( tpDir , DIR_NAME ) ) )
val offsetIndex = spy ( createOffsetIndexForSegmentMetadata ( rlsMetadata , new File ( tpDir , DIR_NAME ) ) )
val spyEntry = spy ( new RemoteIndexCache . Entry ( offsetIndex , timeIndex , txIndex ) )
cache . internalCache . put ( rlsMetadata . remoteLogSegmentId ( ) . id ( ) , spyEntry )
assertCacheSize ( 1 )
var entry : RemoteIndexCache . Entry = null
val latchForCacheRead = new CountDownLatch ( 1 )
val latchForCacheRemove = new CountDownLatch ( 1 )
val latchForTestWait = new CountDownLatch ( 1 )
var markForCleanupCallCount = 0
doAnswer ( ( invocation : InvocationOnMock ) => {
markForCleanupCallCount += 1
if ( markForCleanupCallCount == 1 ) {
// Signal the CacheRead to unblock itself
latchForCacheRead . countDown ( )
// Wait for signal to start renaming the files
latchForCacheRemove . await ( )
// Calling the markForCleanup ( ) actual method to start renaming the files
invocation . callRealMethod ( )
// Signal TestWait to unblock itself so that test can be completed
latchForTestWait . countDown ( )
}
} ) . when ( spyEntry ) . markForCleanup ( )
val removeCache = ( ( ) => {
cache . remove ( rlsMetadata . remoteLogSegmentId ( ) . id ( ) )
} ) : Runnable
val readCache = ( ( ) => {
// Wait for signal to start CacheRead
latchForCacheRead . await ( )
entry = cache . getIndexEntry ( rlsMetadata )
// Signal the CacheRemove to start renaming the files
latchForCacheRemove . countDown ( )
} ) : Runnable
val executor = Executors . newFixedThreadPool ( 2 )
try {
val removeCacheFuture : Future [ _ ] = executor . submit ( removeCache : Runnable )
val readCacheFuture : Future [ _ ] = executor . submit ( readCache : Runnable )
// Verify both tasks are completed without any exception
removeCacheFuture . get ( )
readCacheFuture . get ( )
// Wait for signal to complete the test
latchForTestWait . await ( )
// We can 't determine read thread or remove thread will go first so if ,
// 1. Read thread go first , cache file should not exist and cache size should be zero .
// 2. Remove thread go first , cache file should present and cache size should be one .
// so basically here we are making sure that if cache existed , the cache file should exist ,
// and if cache is non - existed , the cache file should not exist .
if ( getIndexFileFromRemoteCacheDir ( cache , LogFileUtils . INDEX_FILE_SUFFIX ) . isPresent ) {
assertCacheSize ( 1 )
} else {
assertCacheSize ( 0 )
}
} finally {
executor . shutdownNow ( )
}
}
@Test
def testMultipleIndexEntriesExecutionInCorruptException ( ) : Unit = {
reset ( rsm )
@ -679,9 +758,9 @@ class RemoteIndexCacheTest {
@@ -679,9 +758,9 @@ class RemoteIndexCacheTest {
. thenAnswer ( ans => {
val metadata = ans . getArgument [ RemoteLogSegmentMetadata ] ( 0 )
val indexType = ans . getArgument [ IndexType ] ( 1 )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata )
val txnIdx = createTxIndexForSegmentMetadata ( metadata )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata , tpDir )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata , tpDir )
val txnIdx = createTxIndexForSegmentMetadata ( metadata , tpDir )
maybeAppendIndexEntries ( offsetIdx , timeIdx )
// Create corrupted index file
createCorruptTimeIndexOffsetFile ( tpDir )
@ -717,9 +796,9 @@ class RemoteIndexCacheTest {
@@ -717,9 +796,9 @@ class RemoteIndexCacheTest {
. thenAnswer ( ans => {
val metadata = ans . getArgument [ RemoteLogSegmentMetadata ] ( 0 )
val indexType = ans . getArgument [ IndexType ] ( 1 )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata )
val txnIdx = createTxIndexForSegmentMetadata ( metadata )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata , tpDir )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata , tpDir )
val txnIdx = createTxIndexForSegmentMetadata ( metadata , tpDir )
maybeAppendIndexEntries ( offsetIdx , timeIdx )
indexType match {
case IndexType . OFFSET => new FileInputStream ( offsetIdx . file )
@ -764,7 +843,7 @@ class RemoteIndexCacheTest {
@@ -764,7 +843,7 @@ class RemoteIndexCacheTest {
Files . copy ( entry . txnIndex ( ) . file ( ) . toPath ( ) , Paths . get ( Utils . replaceSuffix ( entry . txnIndex ( ) . file ( ) . getPath ( ) , "" , tempSuffix ) ) )
Files . copy ( entry . timeIndex ( ) . file ( ) . toPath ( ) , Paths . get ( Utils . replaceSuffix ( entry . timeIndex ( ) . file ( ) . getPath ( ) , "" , tempSuffix ) ) )
cache . internalCache ( ) . invalidat e( rlsMetadata . remoteLogSegmentId ( ) . id ( ) )
cache . remov e( rlsMetadata . remoteLogSegmentId ( ) . id ( ) )
// wait until entry is marked for deletion
TestUtils . waitUntilTrue ( ( ) => entry . isMarkedForCleanup ,
@ -792,9 +871,9 @@ class RemoteIndexCacheTest {
@@ -792,9 +871,9 @@ class RemoteIndexCacheTest {
. thenAnswer ( ans => {
val metadata = ans . getArgument [ RemoteLogSegmentMetadata ] ( 0 )
val indexType = ans . getArgument [ IndexType ] ( 1 )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata )
val txnIdx = createTxIndexForSegmentMetadata ( metadata )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata , tpDir )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata , tpDir )
val txnIdx = createTxIndexForSegmentMetadata ( metadata , tpDir )
maybeAppendIndexEntries ( offsetIdx , timeIdx )
// Create corrupt index file return from RSM
createCorruptedIndexFile ( testIndexType , tpDir )
@ -839,7 +918,7 @@ class RemoteIndexCacheTest {
@@ -839,7 +918,7 @@ class RemoteIndexCacheTest {
// verify deleted file exists on disk
assertTrue ( getRemoteCacheIndexFileFromDisk ( LogFileUtils . DELETED_FILE_SUFFIX ) . isPresent , s" Deleted Offset index file should be present on disk at ${ remoteIndexCacheDir . toPath } " )
cache . internalCache ( ) . invalidat e( rlsMetadata . remoteLogSegmentId ( ) . id ( ) )
cache . remov e( rlsMetadata . remoteLogSegmentId ( ) . id ( ) )
// wait until entry is marked for deletion
TestUtils . waitUntilTrue ( ( ) => entry . isMarkedForCleanup ,
@ -862,9 +941,9 @@ class RemoteIndexCacheTest {
@@ -862,9 +941,9 @@ class RemoteIndexCacheTest {
= RemoteLogSegmentId . generateNew ( idPartition ) ) : RemoteIndexCache . Entry = {
val rlsMetadata = new RemoteLogSegmentMetadata ( remoteLogSegmentId , baseOffset , lastOffset ,
time . milliseconds ( ) , brokerId , time . milliseconds ( ) , segmentSize , Collections . singletonMap ( 0 , 0L ) )
val timeIndex = spy ( createTimeIndexForSegmentMetadata ( rlsMetadata ) )
val txIndex = spy ( createTxIndexForSegmentMetadata ( rlsMetadata ) )
val offsetIndex = spy ( createOffsetIndexForSegmentMetadata ( rlsMetadata ) )
val timeIndex = spy ( createTimeIndexForSegmentMetadata ( rlsMetadata , tpDir ) )
val txIndex = spy ( createTxIndexForSegmentMetadata ( rlsMetadata , tpDir ) )
val offsetIndex = spy ( createOffsetIndexForSegmentMetadata ( rlsMetadata , tpDir ) )
spy ( new RemoteIndexCache . Entry ( offsetIndex , timeIndex , txIndex ) )
}
@ -892,8 +971,8 @@ class RemoteIndexCacheTest {
@@ -892,8 +971,8 @@ class RemoteIndexCacheTest {
}
}
private def createTxIndexForSegmentMetadata ( metadata : RemoteLogSegmentMetadata ) : TransactionIndex = {
val txnIdxFile = remoteTransactionIndexFile ( tpD ir, metadata )
private def createTxIndexForSegmentMetadata ( metadata : RemoteLogSegmentMetadata , dir : File ) : TransactionIndex = {
val txnIdxFile = remoteTransactionIndexFile ( d ir, metadata )
txnIdxFile . createNewFile ( )
new TransactionIndex ( metadata . startOffset ( ) , txnIdxFile )
}
@ -914,14 +993,14 @@ class RemoteIndexCacheTest {
@@ -914,14 +993,14 @@ class RemoteIndexCacheTest {
return new TransactionIndex ( 100L , txnIdxFile )
}
private def createTimeIndexForSegmentMetadata ( metadata : RemoteLogSegmentMetadata ) : TimeIndex = {
private def createTimeIndexForSegmentMetadata ( metadata : RemoteLogSegmentMetadata , dir : File ) : TimeIndex = {
val maxEntries = ( metadata . endOffset ( ) - metadata . startOffset ( ) ) . asInstanceOf [ Int ]
new TimeIndex ( remoteTimeIndexFile ( tpD ir, metadata ) , metadata . startOffset ( ) , maxEntries * 12 )
new TimeIndex ( remoteTimeIndexFile ( d ir, metadata ) , metadata . startOffset ( ) , maxEntries * 12 )
}
private def createOffsetIndexForSegmentMetadata ( metadata : RemoteLogSegmentMetadata ) = {
private def createOffsetIndexForSegmentMetadata ( metadata : RemoteLogSegmentMetadata , dir : File ) = {
val maxEntries = ( metadata . endOffset ( ) - metadata . startOffset ( ) ) . asInstanceOf [ Int ]
new OffsetIndex ( remoteOffsetIndexFile ( tpD ir, metadata ) , metadata . startOffset ( ) , maxEntries * 8 )
new OffsetIndex ( remoteOffsetIndexFile ( d ir, metadata ) , metadata . startOffset ( ) , maxEntries * 8 )
}
private def generateRemoteLogSegmentMetadata ( size : Int ,
@ -969,9 +1048,9 @@ class RemoteIndexCacheTest {
@@ -969,9 +1048,9 @@ class RemoteIndexCacheTest {
. thenAnswer ( ans => {
val metadata = ans . getArgument [ RemoteLogSegmentMetadata ] ( 0 )
val indexType = ans . getArgument [ IndexType ] ( 1 )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata )
val txnIdx = createTxIndexForSegmentMetadata ( metadata )
val offsetIdx = createOffsetIndexForSegmentMetadata ( metadata , tpDir )
val timeIdx = createTimeIndexForSegmentMetadata ( metadata , tpDir )
val txnIdx = createTxIndexForSegmentMetadata ( metadata , tpDir )
maybeAppendIndexEntries ( offsetIdx , timeIdx )
indexType match {
case IndexType . OFFSET => new FileInputStream ( offsetIdx . file )