@ -74,6 +74,7 @@ import org.apache.kafka.test.TestUtils;
@@ -74,6 +74,7 @@ import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.BeforeEach ;
import org.junit.jupiter.api.Test ;
import org.junit.jupiter.params.ParameterizedTest ;
import org.junit.jupiter.params.provider.CsvSource ;
import org.junit.jupiter.params.provider.ValueSource ;
import org.mockito.ArgumentCaptor ;
import org.mockito.InOrder ;
@ -105,6 +106,7 @@ import java.util.Set;
@@ -105,6 +106,7 @@ import java.util.Set;
import java.util.TreeMap ;
import java.util.concurrent.CompletableFuture ;
import java.util.concurrent.CountDownLatch ;
import java.util.concurrent.ExecutionException ;
import java.util.concurrent.atomic.AtomicLong ;
import java.util.function.BiConsumer ;
import java.util.function.Supplier ;
@ -189,6 +191,7 @@ public class RemoteLogManagerTest {
@@ -189,6 +191,7 @@ public class RemoteLogManagerTest {
return epochs ;
}
} ;
private final AtomicLong currentLogStartOffset = new AtomicLong ( 0L ) ;
private final UnifiedLog mockLog = mock ( UnifiedLog . class ) ;
@ -204,7 +207,7 @@ public class RemoteLogManagerTest {
@@ -204,7 +207,7 @@ public class RemoteLogManagerTest {
kafka . utils . TestUtils . clearYammerMetrics ( ) ;
remoteLogManager = new RemoteLogManager ( remoteLogManagerConfig , brokerId , logDir , clusterId , time ,
tp - > Optional . of ( mockLog ) ,
( topicPartition , offset ) - > { } ,
( topicPartition , offset ) - > currentLogStartOffset . set ( offset ) ,
brokerTopicStats ) {
public RemoteStorageManager createRemoteStorageManager ( ) {
return remoteStorageManager ;
@ -1259,14 +1262,15 @@ public class RemoteLogManagerTest {
@@ -1259,14 +1262,15 @@ public class RemoteLogManagerTest {
segmentEpochs7 ) , logEndOffset , leaderEpochToStartOffset ) ) ;
// Test a remote segment having larger end offset than the log end offset
TreeMap < Integer , Long > segmentEpochs8 = new TreeMap < > ( ) ;
segmentEpochs8 . put ( 1 , 15L ) ;
segmentEpochs8 . put ( 2 , 20L ) ;
assertFalse ( RemoteLogManager . isRemoteSegmentWithinLeaderEpochs ( createRemoteLogSegmentMetadata (
15 ,
95 , // larger than log end offset
segmentEpochs8 ) , logEndOffset , leaderEpochToStartOffset ) ) ;
leaderEpochToStartOffset ) , logEndOffset , leaderEpochToStartOffset ) ) ;
assertFalse ( RemoteLogManager . isRemoteSegmentWithinLeaderEpochs ( createRemoteLogSegmentMetadata (
15 ,
90 , // equal to the log end offset
leaderEpochToStartOffset ) , logEndOffset , leaderEpochToStartOffset ) ) ;
// Test whether a segment's first offset is earlier to the respective epoch's start offset
TreeMap < Integer , Long > segmentEpochs9 = new TreeMap < > ( ) ;
@ -1521,188 +1525,212 @@ public class RemoteLogManagerTest {
@@ -1521,188 +1525,212 @@ public class RemoteLogManagerTest {
}
}
@Test
public void testDeleteRetentionSizeBreachingSegments ( ) throws RemoteStorageException , IOException {
AtomicLong logStartOffset = new AtomicLong ( 0 ) ;
try ( RemoteLogManager remoteLogManager = new RemoteLogManager ( remoteLogManagerConfig , brokerId , logDir , clusterId , time ,
tp - > Optional . of ( mockLog ) ,
( topicPartition , offset ) - > logStartOffset . set ( offset ) ,
brokerTopicStats ) {
public RemoteStorageManager createRemoteStorageManager ( ) {
return remoteStorageManager ;
}
public RemoteLogMetadataManager createRemoteLogMetadataManager ( ) {
return remoteLogMetadataManager ;
}
} ) {
RemoteLogManager . RLMTask task = remoteLogManager . new RLMTask ( leaderTopicIdPartition , 128 ) ;
task . convertToLeader ( 0 ) ;
when ( mockLog . topicPartition ( ) ) . thenReturn ( leaderTopicIdPartition . topicPartition ( ) ) ;
when ( mockLog . logEndOffset ( ) ) . thenReturn ( 200L ) ;
List < EpochEntry > epochEntries = Collections . singletonList ( epochEntry0 ) ;
List < RemoteLogSegmentMetadata > remoteLogSegmentMetadatas = listRemoteLogSegmentMetadata ( leaderTopicIdPartition , 2 , 100 , 1024 , epochEntries ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition , 0 ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) ) ;
checkpoint . write ( epochEntries ) ;
LeaderEpochFileCache cache = new LeaderEpochFileCache ( tp , checkpoint ) ;
when ( mockLog . leaderEpochCache ( ) ) . thenReturn ( Option . apply ( cache ) ) ;
@ParameterizedTest ( name = "testDeletionOnRetentionBreachedSegments retentionSize={0} retentionMs={1}" )
@CsvSource ( value = { "0, -1" , "-1, 0" } )
public void testDeletionOnRetentionBreachedSegments ( long retentionSize ,
long retentionMs )
throws RemoteStorageException , ExecutionException , InterruptedException {
Map < String , Long > logProps = new HashMap < > ( ) ;
logProps . put ( "retention.bytes" , retentionSize ) ;
logProps . put ( "retention.ms" , retentionMs ) ;
LogConfig mockLogConfig = new LogConfig ( logProps ) ;
when ( mockLog . config ( ) ) . thenReturn ( mockLogConfig ) ;
List < EpochEntry > epochEntries = Collections . singletonList ( epochEntry0 ) ;
checkpoint . write ( epochEntries ) ;
LeaderEpochFileCache cache = new LeaderEpochFileCache ( tp , checkpoint ) ;
when ( mockLog . leaderEpochCache ( ) ) . thenReturn ( Option . apply ( cache ) ) ;
Map < String , Long > logProps = new HashMap < > ( ) ;
logProps . put ( "retention.bytes" , 0L ) ;
logProps . put ( "retention.ms" , - 1L ) ;
LogConfig mockLogConfig = new LogConfig ( logProps ) ;
when ( mockLog . config ( ) ) . thenReturn ( mockLogConfig ) ;
when ( mockLog . topicPartition ( ) ) . thenReturn ( leaderTopicIdPartition . topicPartition ( ) ) ;
when ( mockLog . logEndOffset ( ) ) . thenReturn ( 200L ) ;
when ( remoteLogMetadataManager . updateRemoteLogSegmentMetadata ( any ( RemoteLogSegmentMetadataUpdate . class ) ) )
. thenAnswer ( answer - > CompletableFuture . runAsync ( ( ) - > { } ) ) ;
List < RemoteLogSegmentMetadata > metadataList =
listRemoteLogSegmentMetadata ( leaderTopicIdPartition , 2 , 100 , 1024 , epochEntries ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition ) )
. thenReturn ( metadataList . iterator ( ) ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition , 0 ) )
. thenAnswer ( ans - > metadataList . iterator ( ) ) ;
when ( remoteLogMetadataManager . updateRemoteLogSegmentMetadata ( any ( RemoteLogSegmentMetadataUpdate . class ) ) )
. thenReturn ( CompletableFuture . runAsync ( ( ) - > { } ) ) ;
task . run ( ) ;
RemoteLogManager . RLMTask task = remoteLogManager . new RLMTask ( leaderTopicIdPartition , 128 ) ;
task . convertToLeader ( 0 ) ;
task . cleanupExpiredRemoteLogSegments ( ) ;
assertEquals ( 200L , logStartOffset . get ( ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( remoteLogSegmentMetadatas . get ( 0 ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( remoteLogSegmentMetadatas . get ( 1 ) ) ;
}
assertEquals ( 200L , currentLogStartOffset . get ( ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( metadataList . get ( 0 ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( metadataList . get ( 1 ) ) ;
}
@Test
public void testDeleteRetentionMsBreachingSegments ( ) throws RemoteStorageException , IOException {
AtomicLong logStartOffset = new AtomicLong ( 0 ) ;
try ( RemoteLogManager remoteLogManager = new RemoteLogManager ( remoteLogManagerConfig , brokerId , logDir , clusterId , time ,
tp - > Optional . of ( mockLog ) ,
( topicPartition , offset ) - > logStartOffset . set ( offset ) ,
brokerTopicStats ) {
public RemoteStorageManager createRemoteStorageManager ( ) {
return remoteStorageManager ;
}
public RemoteLogMetadataManager createRemoteLogMetadataManager ( ) {
return remoteLogMetadataManager ;
}
} ) {
RemoteLogManager . RLMTask task = remoteLogManager . new RLMTask ( leaderTopicIdPartition , 128 ) ;
task . convertToLeader ( 0 ) ;
public void testDeleteRetentionMsBeingCancelledBeforeSecondDelete ( ) throws RemoteStorageException , ExecutionException , InterruptedException {
RemoteLogManager . RLMTask leaderTask = remoteLogManager . new RLMTask ( leaderTopicIdPartition , 128 ) ;
leaderTask . convertToLeader ( 0 ) ;
when ( mockLog . topicPartition ( ) ) . thenReturn ( leaderTopicIdPartition . topicPartition ( ) ) ;
when ( mockLog . logEndOffset ( ) ) . thenReturn ( 200L ) ;
when ( mockLog . topicPartition ( ) ) . thenReturn ( leaderTopicIdPartition . topicPartition ( ) ) ;
when ( mockLog . logEndOffset ( ) ) . thenReturn ( 200L ) ;
List < EpochEntry > epochEntries = Collections . singletonList ( epochEntry0 ) ;
List < EpochEntry > epochEntries = Collections . singletonList ( epochEntry0 ) ;
List < RemoteLogSegmentMetadata > remoteLogSegmentMetadatas = listRemoteLogSegmentMetadata ( leaderTopicIdPartition , 2 , 100 , 1024 , epochEntries ) ;
List < RemoteLogSegmentMetadata > metadataList =
listRemoteLogSegmentMetadata ( leaderTopicIdPartition , 2 , 100 , 1024 , epochEntries ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition ) )
. thenReturn ( metadataList . iterator ( ) ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition , 0 ) )
. thenAnswer ( ans - > metadataList . iterator ( ) ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition , 0 ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) ) ;
checkpoint . write ( epochEntries ) ;
LeaderEpochFileCache cache = new LeaderEpochFileCache ( tp , checkpoint ) ;
when ( mockLog . leaderEpochCache ( ) ) . thenReturn ( Option . apply ( cache ) ) ;
checkpoint . write ( epochEntries ) ;
LeaderEpochFileCache cache = new LeaderEpochFileCache ( tp , checkpoint ) ;
when ( mockLog . leaderEpochCache ( ) ) . thenReturn ( Option . apply ( cache ) ) ;
Map < String , Long > logProps = new HashMap < > ( ) ;
logProps . put ( "retention.bytes" , - 1L ) ;
logProps . put ( "retention.ms" , 0L ) ;
LogConfig mockLogConfig = new LogConfig ( logProps ) ;
when ( mockLog . config ( ) ) . thenReturn ( mockLogConfig ) ;
when ( remoteLogMetadataManager . updateRemoteLogSegmentMetadata ( any ( RemoteLogSegmentMetadataUpdate . class ) ) )
. thenAnswer ( answer - > {
// cancel the task so that we don't delete the second segment
leaderTask . cancel ( ) ;
return CompletableFuture . runAsync ( ( ) - > {
} ) ;
} ) ;
Map < String , Long > logProps = new HashMap < > ( ) ;
logProps . put ( "retention.bytes" , - 1L ) ;
logProps . put ( "retention.ms" , 0L ) ;
LogConfig mockLogConfig = new LogConfig ( logProps ) ;
when ( mockLog . config ( ) ) . thenReturn ( mockLogConfig ) ;
leaderTask . cleanupExpiredRemoteLogSegments ( ) ;
when ( remoteLogMetadataManager . updateRemoteLogSegmentMetadata ( any ( RemoteLogSegmentMetadataUpdate . class ) ) )
. thenAnswer ( answer - > CompletableFuture . runAsync ( ( ) - > { } ) ) ;
assertEquals ( 200L , currentLogStartOffset . get ( ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( metadataList . get ( 0 ) ) ;
verify ( remoteStorageManager , never ( ) ) . deleteLogSegmentData ( metadataList . get ( 1 ) ) ;
task . run ( ) ;
// test that the 2nd log segment will be deleted by the new leader
RemoteLogManager . RLMTask newLeaderTask = remoteLogManager . new RLMTask ( followerTopicIdPartition , 128 ) ;
newLeaderTask . convertToLeader ( 1 ) ;
assertEquals ( 200L , logStartOffset . get ( ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( remoteLogSegmentMetadatas . get ( 0 ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( remoteLogSegmentMetadatas . get ( 1 ) ) ;
}
}
Iterator < RemoteLogSegmentMetadata > firstIterator = metadataList . iterator ( ) ;
firstIterator . next ( ) ;
Iterator < RemoteLogSegmentMetadata > secondIterator = metadataList . iterator ( ) ;
secondIterator . next ( ) ;
Iterator < RemoteLogSegmentMetadata > thirdIterator = metadataList . iterator ( ) ;
thirdIterator . next ( ) ;
@Test
public void testDeleteRetentionMsBeingCancelledBeforeSecondDelete ( ) throws RemoteStorageException , IOException {
AtomicLong logStartOffset = new AtomicLong ( 0 ) ;
try ( RemoteLogManager remoteLogManager = new RemoteLogManager ( remoteLogManagerConfig , brokerId , logDir , clusterId , time ,
tp - > Optional . of ( mockLog ) ,
( topicPartition , offset ) - > logStartOffset . set ( offset ) ,
brokerTopicStats ) {
public RemoteStorageManager createRemoteStorageManager ( ) {
return remoteStorageManager ;
}
public RemoteLogMetadataManager createRemoteLogMetadataManager ( ) {
return remoteLogMetadataManager ;
}
} ) {
RemoteLogManager . RLMTask leaderTask = remoteLogManager . new RLMTask ( leaderTopicIdPartition , 128 ) ;
leaderTask . convertToLeader ( 0 ) ;
when ( mockLog . topicPartition ( ) ) . thenReturn ( leaderTopicIdPartition . topicPartition ( ) ) ;
when ( mockLog . logEndOffset ( ) ) . thenReturn ( 200L ) ;
List < EpochEntry > epochEntries = Collections . singletonList ( epochEntry0 ) ;
List < RemoteLogSegmentMetadata > remoteLogSegmentMetadatas = listRemoteLogSegmentMetadata ( leaderTopicIdPartition , 2 , 100 , 1024 , epochEntries ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition , 0 ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) )
. thenReturn ( remoteLogSegmentMetadatas . iterator ( ) ) ;
checkpoint . write ( epochEntries ) ;
LeaderEpochFileCache cache = new LeaderEpochFileCache ( tp , checkpoint ) ;
when ( mockLog . leaderEpochCache ( ) ) . thenReturn ( Option . apply ( cache ) ) ;
Map < String , Long > logProps = new HashMap < > ( ) ;
logProps . put ( "retention.bytes" , - 1L ) ;
logProps . put ( "retention.ms" , 0L ) ;
LogConfig mockLogConfig = new LogConfig ( logProps ) ;
when ( mockLog . config ( ) ) . thenReturn ( mockLogConfig ) ;
when ( remoteLogMetadataManager . updateRemoteLogSegmentMetadata ( any ( RemoteLogSegmentMetadataUpdate . class ) ) )
. thenAnswer ( answer - > {
// cancel the task so that we don't delete the second segment
leaderTask . cancel ( ) ;
return CompletableFuture . runAsync ( ( ) - > {
} ) ;
} ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( followerTopicIdPartition ) )
. thenReturn ( firstIterator ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( followerTopicIdPartition , 0 ) )
. thenReturn ( secondIterator )
. thenReturn ( thirdIterator ) ;
leaderTask . run ( ) ;
when ( remoteLogMetadataManager . updateRemoteLogSegmentMetadata ( any ( RemoteLogSegmentMetadataUpdate . class ) ) )
. thenAnswer ( answer - > CompletableFuture . runAsync ( ( ) - > { } ) ) ;
assertEquals ( 200L , logStartOffset . get ( ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( remoteLogSegmentMetadatas . get ( 0 ) ) ;
verify ( remoteStorageManager , never ( ) ) . deleteLogSegmentData ( remoteLogSegmentMetadatas . get ( 1 ) ) ;
newLeaderTask . cleanupExpiredRemoteLogSegments ( ) ;
// test that the 2nd log segment will be deleted by the new leader
RemoteLogManager . RLMTask newLeaderTask = remoteLogManager . new RLMTask ( followerTopicIdPartition , 128 ) ;
newLeaderTask . convertToLeader ( 1 ) ;
assertEquals ( 200L , currentLogStartOffset . get ( ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( metadataList . get ( 0 ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( metadataList . get ( 1 ) ) ;
}
Iterator < RemoteLogSegmentMetadata > firstIterator = remoteLogSegmentMetadatas . iterator ( ) ;
firstIterator . next ( ) ;
Iterator < RemoteLogSegmentMetadata > secondIterator = remoteLogSegmentMetadatas . iterator ( ) ;
secondIterator . next ( ) ;
Iterator < RemoteLogSegmentMetadata > thirdIterator = remoteLogSegmentMetadatas . iterator ( ) ;
thirdIterator . next ( ) ;
@ParameterizedTest ( name = "testDeleteLogSegmentDueToRetentionSizeBreach segmentCount={0} deletableSegmentCount={1}" )
@CsvSource ( value = { "50, 0" , "50, 1" , "50, 23" , "50, 50" } )
public void testDeleteLogSegmentDueToRetentionSizeBreach ( int segmentCount ,
int deletableSegmentCount )
throws RemoteStorageException , ExecutionException , InterruptedException {
int recordsPerSegment = 100 ;
int segmentSize = 1024 ;
List < EpochEntry > epochEntries = Arrays . asList (
new EpochEntry ( 0 , 0L ) ,
new EpochEntry ( 1 , 20L ) ,
new EpochEntry ( 3 , 50L ) ,
new EpochEntry ( 4 , 100L )
) ;
checkpoint . write ( epochEntries ) ;
LeaderEpochFileCache cache = new LeaderEpochFileCache ( tp , checkpoint ) ;
int currentLeaderEpoch = epochEntries . get ( epochEntries . size ( ) - 1 ) . epoch ;
long localLogSegmentsSize = 512L ;
long retentionSize = ( ( long ) segmentCount - deletableSegmentCount ) * segmentSize + localLogSegmentsSize ;
Map < String , Long > logProps = new HashMap < > ( ) ;
logProps . put ( "retention.bytes" , retentionSize ) ;
logProps . put ( "retention.ms" , - 1L ) ;
LogConfig mockLogConfig = new LogConfig ( logProps ) ;
when ( mockLog . config ( ) ) . thenReturn ( mockLogConfig ) ;
long localLogStartOffset = ( long ) segmentCount * recordsPerSegment ;
long logEndOffset = ( ( long ) segmentCount * recordsPerSegment ) + 1 ;
when ( mockLog . leaderEpochCache ( ) ) . thenReturn ( Option . apply ( cache ) ) ;
when ( mockLog . localLogStartOffset ( ) ) . thenReturn ( localLogStartOffset ) ;
when ( mockLog . logEndOffset ( ) ) . thenReturn ( logEndOffset ) ;
when ( mockLog . onlyLocalLogSegmentsSize ( ) ) . thenReturn ( localLogSegmentsSize ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( followerTopicIdPartition ) )
. thenReturn ( firstIterator ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( followerTopicIdPartition , 0 ) )
. thenReturn ( secondIterator )
. thenReturn ( thirdIterator ) ;
List < RemoteLogSegmentMetadata > segmentMetadataList = listRemoteLogSegmentMetadata (
leaderTopicIdPartition , segmentCount , recordsPerSegment , segmentSize , epochEntries ) ;
verifyDeleteLogSegment ( segmentMetadataList , deletableSegmentCount , currentLeaderEpoch ) ;
}
when ( remoteLogMetadataManager . updateRemoteLogSegmentMetadata ( any ( RemoteLogSegmentMetadataUpdate . class ) ) )
. thenAnswer ( answer - > CompletableFuture . runAsync ( ( ) - > { } ) ) ;
@ParameterizedTest ( name = "testDeleteLogSegmentDueToRetentionTimeBreach segmentCount={0} deletableSegmentCount={1}" )
@CsvSource ( value = { "50, 0" , "50, 1" , "50, 23" , "50, 50" } )
public void testDeleteLogSegmentDueToRetentionTimeBreach ( int segmentCount ,
int deletableSegmentCount )
throws RemoteStorageException , ExecutionException , InterruptedException {
int recordsPerSegment = 100 ;
int segmentSize = 1024 ;
List < EpochEntry > epochEntries = Arrays . asList (
new EpochEntry ( 0 , 0L ) ,
new EpochEntry ( 1 , 20L ) ,
new EpochEntry ( 3 , 50L ) ,
new EpochEntry ( 4 , 100L )
) ;
checkpoint . write ( epochEntries ) ;
LeaderEpochFileCache cache = new LeaderEpochFileCache ( tp , checkpoint ) ;
int currentLeaderEpoch = epochEntries . get ( epochEntries . size ( ) - 1 ) . epoch ;
long localLogSegmentsSize = 512L ;
long retentionSize = - 1L ;
Map < String , Long > logProps = new HashMap < > ( ) ;
logProps . put ( "retention.bytes" , retentionSize ) ;
logProps . put ( "retention.ms" , 1L ) ;
LogConfig mockLogConfig = new LogConfig ( logProps ) ;
when ( mockLog . config ( ) ) . thenReturn ( mockLogConfig ) ;
long localLogStartOffset = ( long ) segmentCount * recordsPerSegment ;
long logEndOffset = ( ( long ) segmentCount * recordsPerSegment ) + 1 ;
when ( mockLog . leaderEpochCache ( ) ) . thenReturn ( Option . apply ( cache ) ) ;
when ( mockLog . localLogStartOffset ( ) ) . thenReturn ( localLogStartOffset ) ;
when ( mockLog . logEndOffset ( ) ) . thenReturn ( logEndOffset ) ;
when ( mockLog . onlyLocalLogSegmentsSize ( ) ) . thenReturn ( localLogSegmentsSize ) ;
newLeaderTask . run ( ) ;
List < RemoteLogSegmentMetadata > segmentMetadataList = listRemoteLogSegmentMetadataByTime (
leaderTopicIdPartition , segmentCount , deletableSegmentCount , recordsPerSegment , segmentSize , epochEntries ) ;
verifyDeleteLogSegment ( segmentMetadataList , deletableSegmentCount , currentLeaderEpoch ) ;
}
assertEquals ( 200L , logStartOffset . get ( ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( remoteLogSegmentMetadatas . get ( 0 ) ) ;
verify ( remoteStorageManager ) . deleteLogSegmentData ( remoteLogSegmentMetadatas . get ( 1 ) ) ;
private void verifyDeleteLogSegment ( List < RemoteLogSegmentMetadata > segmentMetadataList ,
int deletableSegmentCount ,
int currentLeaderEpoch )
throws RemoteStorageException , ExecutionException , InterruptedException {
when ( remoteLogMetadataManager . listRemoteLogSegments ( leaderTopicIdPartition ) )
. thenReturn ( segmentMetadataList . iterator ( ) ) ;
when ( remoteLogMetadataManager . listRemoteLogSegments ( eq ( leaderTopicIdPartition ) , anyInt ( ) ) )
. thenAnswer ( invocation - > {
int leaderEpoch = invocation . getArgument ( 1 ) ;
return segmentMetadataList . stream ( )
. filter ( segmentMetadata - > segmentMetadata . segmentLeaderEpochs ( ) . containsKey ( leaderEpoch ) )
. iterator ( ) ;
} ) ;
when ( remoteLogMetadataManager . updateRemoteLogSegmentMetadata ( any ( RemoteLogSegmentMetadataUpdate . class ) ) )
. thenAnswer ( answer - > CompletableFuture . runAsync ( ( ) - > { } ) ) ;
RemoteLogManager . RLMTask task = remoteLogManager . new RLMTask ( leaderTopicIdPartition , 128 ) ;
task . convertToLeader ( currentLeaderEpoch ) ;
task . cleanupExpiredRemoteLogSegments ( ) ;
ArgumentCaptor < RemoteLogSegmentMetadata > deletedMetadataCapture = ArgumentCaptor . forClass ( RemoteLogSegmentMetadata . class ) ;
verify ( remoteStorageManager , times ( deletableSegmentCount ) ) . deleteLogSegmentData ( deletedMetadataCapture . capture ( ) ) ;
if ( deletableSegmentCount > 0 ) {
List < RemoteLogSegmentMetadata > deletedMetadataList = deletedMetadataCapture . getAllValues ( ) ;
RemoteLogSegmentMetadata expectedEndMetadata = segmentMetadataList . get ( deletableSegmentCount - 1 ) ;
assertEquals ( segmentMetadataList . get ( 0 ) , deletedMetadataList . get ( 0 ) ) ;
assertEquals ( expectedEndMetadata , deletedMetadataList . get ( deletedMetadataList . size ( ) - 1 ) ) ;
assertEquals ( currentLogStartOffset . get ( ) , expectedEndMetadata . endOffset ( ) + 1 ) ;
}
}
@ -1718,9 +1746,22 @@ public class RemoteLogManagerTest {
@@ -1718,9 +1746,22 @@ public class RemoteLogManagerTest {
int recordsPerSegment ,
int segmentSize ,
List < EpochEntry > epochEntries ) {
return listRemoteLogSegmentMetadataByTime (
topicIdPartition , segmentCount , 0 , recordsPerSegment , segmentSize , epochEntries ) ;
}
private List < RemoteLogSegmentMetadata > listRemoteLogSegmentMetadataByTime ( TopicIdPartition topicIdPartition ,
int segmentCount ,
int deletableSegmentCount ,
int recordsPerSegment ,
int segmentSize ,
List < EpochEntry > epochEntries ) {
List < RemoteLogSegmentMetadata > segmentMetadataList = new ArrayList < > ( ) ;
for ( int idx = 0 ; idx < segmentCount ; idx + + ) {
long timestamp = time . milliseconds ( ) ;
if ( idx < deletableSegmentCount ) {
timestamp = time . milliseconds ( ) - 1 ;
}
long startOffset = ( long ) idx * recordsPerSegment ;
long endOffset = startOffset + recordsPerSegment - 1 ;
List < EpochEntry > localTotalEpochEntries = epochEntries . isEmpty ( ) ? totalEpochEntries : epochEntries ;