@ -84,7 +84,7 @@ class LogCleanerTest {
val segments = log . logSegments . take ( 3 ) . toSeq
val segments = log . logSegments . take ( 3 ) . toSeq
val stats = new CleanerStats ( )
val stats = new CleanerStats ( )
val expectedBytesRead = segments . map ( _ . size ) . sum
val expectedBytesRead = segments . map ( _ . size ) . sum
cleaner . cleanSegments ( log , segments , map , 0L , stats )
cleaner . cleanSegments ( log , segments , map , 0L , stats , new CleanedTransactionMetadata )
val shouldRemain = LogTest . keysInLog ( log ) . filter ( ! keys . contains ( _ ) )
val shouldRemain = LogTest . keysInLog ( log ) . filter ( ! keys . contains ( _ ) )
assertEquals ( shouldRemain , LogTest . keysInLog ( log ) )
assertEquals ( shouldRemain , LogTest . keysInLog ( log ) )
assertEquals ( expectedBytesRead , stats . bytesRead )
assertEquals ( expectedBytesRead , stats . bytesRead )
@ -151,7 +151,7 @@ class LogCleanerTest {
val segments = log . logSegments ( 0 , log . activeSegment . baseOffset ) . toSeq
val segments = log . logSegments ( 0 , log . activeSegment . baseOffset ) . toSeq
val stats = new CleanerStats ( )
val stats = new CleanerStats ( )
cleaner . buildOffsetMap ( log , 0 , log . activeSegment . baseOffset , offsetMap , stats )
cleaner . buildOffsetMap ( log , 0 , log . activeSegment . baseOffset , offsetMap , stats )
cleaner . cleanSegments ( log , segments , offsetMap , 0L , stats )
cleaner . cleanSegments ( log , segments , offsetMap , 0L , stats , new CleanedTransactionMetadata )
// Validate based on the file name that log segment file is renamed exactly once for async deletion
// Validate based on the file name that log segment file is renamed exactly once for async deletion
assertEquals ( expectedFileName , firstLogFile . file ( ) . getPath )
assertEquals ( expectedFileName , firstLogFile . file ( ) . getPath )
@ -432,37 +432,37 @@ class LogCleanerTest {
log . roll ( )
log . roll ( )
// first time through the records are removed
// first time through the records are removed
// Expected State : [ { Producer1 : EmptyBatch } , { Producer2 : EmptyBatch } , { Producer2 : Commit } , { 2 } , { 3 } ]
// Expected State : [ { Producer1 : EmptyBatch } , { Producer2 : EmptyBatch } , { Producer2 : Commit } , { 2 } , { 3 } , { Producer1 : Commit } ]
var dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , 0L , 100L ) , deleteHorizonMs = Long . MaxValue ) . _1
var dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , 0L , 100L ) , deleteHorizonMs = Long . MaxValue ) . _1
assertEquals ( List ( 2 , 3 ) , LogTest . keysInLog ( log ) )
assertEquals ( List ( 2 , 3 ) , LogTest . keysInLog ( log ) )
assertEquals ( List ( 4 , 5 , 6 ) , offsetsInLog ( log ) )
assertEquals ( List ( 4 , 5 , 6 , 7 ) , offsetsInLog ( log ) )
assertEquals ( List ( 1 , 3 , 4 , 5 , 6 ) , lastOffsetsPerBatchInLog ( log ) )
assertEquals ( List ( 1 , 3 , 4 , 5 , 6 , 7 ) , lastOffsetsPerBatchInLog ( log ) )
// the empty batch remains if cleaned again because it still holds the last sequence
// the empty batch remains if cleaned again because it still holds the last sequence
// Expected State : [ { Producer1 : EmptyBatch } , { Producer2 : EmptyBatch } , { Producer2 : Commit } , { 2 } , { 3 } ]
// Expected State : [ { Producer1 : EmptyBatch } , { Producer2 : EmptyBatch } , { Producer2 : Commit } , { 2 } , { 3 } , { Producer1 : Commit } ]
dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = Long . MaxValue ) . _1
dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = Long . MaxValue ) . _1
assertEquals ( List ( 2 , 3 ) , LogTest . keysInLog ( log ) )
assertEquals ( List ( 2 , 3 ) , LogTest . keysInLog ( log ) )
assertEquals ( List ( 4 , 5 , 6 ) , offsetsInLog ( log ) )
assertEquals ( List ( 4 , 5 , 6 , 7 ) , offsetsInLog ( log ) )
assertEquals ( List ( 1 , 3 , 4 , 5 , 6 ) , lastOffsetsPerBatchInLog ( log ) )
assertEquals ( List ( 1 , 3 , 4 , 5 , 6 , 7 ) , lastOffsetsPerBatchInLog ( log ) )
// append a new record from the producer to allow cleaning of the empty batch
// append a new record from the producer to allow cleaning of the empty batch
// [ { Producer1 : EmptyBatch } , { Producer2 : EmptyBatch } , { Producer2 : Commit } , { 2 } , { 3 } ] [ { Producer2 : 1 } , { Producer2 : Commit } ]
// [ { Producer1 : EmptyBatch } , { Producer2 : EmptyBatch } , { Producer2 : Commit } , { 2 } , { 3 } , { Producer1 : Commit } , { Producer2 : 1 } , { Producer2 : Commit } ]
// { 1 } , { 3 } , { 4 } , { 5 } , { 6 } , { 8 } , { 9 } ==> Offsets
// { 1 } , { 3 } , { 4 } , { 5 } , { 6 } , { 7 } , { 8 } , { 9 } ==> Offsets
producer2 ( Seq ( 1 ) ) // offset 8
producer2 ( Seq ( 1 ) ) // offset 8
log . appendAsLeader ( commitMarker ( 2L , producerEpoch ) , leaderEpoch = 0 , isFromClient = false ) // offset 9
log . appendAsLeader ( commitMarker ( 2L , producerEpoch ) , leaderEpoch = 0 , isFromClient = false ) // offset 9
log . roll ( )
log . roll ( )
// Expected State : [ { Producer1 : EmptyBatch } , { Producer2 : Commit } , { 2 } , { 3 } , { Producer2 : 1 } , { Producer2 : Commit } ]
// Expected State : [ { Producer1 : EmptyBatch } , { Producer2 : Commit } , { 2 } , { 3 } , { Producer1 : Commit } , { Producer 2 : 1 } , { Producer2 : Commit } ]
dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = Long . MaxValue ) . _1
dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = Long . MaxValue ) . _1
assertEquals ( List ( 2 , 3 , 1 ) , LogTest . keysInLog ( log ) )
assertEquals ( List ( 2 , 3 , 1 ) , LogTest . keysInLog ( log ) )
assertEquals ( List ( 4 , 5 , 6 , 8 , 9 ) , offsetsInLog ( log ) )
assertEquals ( List ( 4 , 5 , 6 , 7 , 8 , 9 ) , offsetsInLog ( log ) )
assertEquals ( List ( 1 , 4 , 5 , 6 , 8 , 9 ) , lastOffsetsPerBatchInLog ( log ) )
assertEquals ( List ( 1 , 4 , 5 , 6 , 7 , 8 , 9 ) , lastOffsetsPerBatchInLog ( log ) )
// Expected State : [ { Producer1 : EmptyBatch } , { 2 } , { 3 } , { Producer2 : 1 } , { Producer2 : Commit } ]
// Expected State : [ { Producer1 : EmptyBatch } , { 2 } , { 3 } , { Producer1 : Commit } , { Producer 2 : 1 } , { Producer2 : Commit } ]
dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = Long . MaxValue ) . _1
dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = Long . MaxValue ) . _1
assertEquals ( List ( 2 , 3 , 1 ) , LogTest . keysInLog ( log ) )
assertEquals ( List ( 2 , 3 , 1 ) , LogTest . keysInLog ( log ) )
assertEquals ( List ( 5 , 6 , 8 , 9 ) , offsetsInLog ( log ) )
assertEquals ( List ( 5 , 6 , 7 , 8 , 9 ) , offsetsInLog ( log ) )
assertEquals ( List ( 1 , 5 , 6 , 8 , 9 ) , lastOffsetsPerBatchInLog ( log ) )
assertEquals ( List ( 1 , 5 , 6 , 7 , 8 , 9 ) , lastOffsetsPerBatchInLog ( log ) )
}
}
@Test
@Test
@ -496,6 +496,59 @@ class LogCleanerTest {
assertEquals ( List ( 0 , 1 , 2 ) , lastOffsetsPerBatchInLog ( log ) )
assertEquals ( List ( 0 , 1 , 2 ) , lastOffsetsPerBatchInLog ( log ) )
}
}
@Test
def testCommittedTransactionSpanningSegments ( ) : Unit = {
val tp = new TopicPartition ( "test" , 0 )
val cleaner = makeCleaner ( Int . MaxValue )
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , 128 : java . lang . Integer )
val log = makeLog ( config = LogConfig . fromProps ( logConfig . originals , logProps ) )
val producerEpoch = 0. toShort
val producerId = 1L
val appendTransaction = appendTransactionalAsLeader ( log , producerId , producerEpoch )
appendTransaction ( Seq ( 1 ) )
log . roll ( )
log . appendAsLeader ( commitMarker ( producerId , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
log . roll ( )
// Both the record and the marker should remain after cleaning
cleaner . doClean ( LogToClean ( tp , log , 0L , 100L ) , deleteHorizonMs = Long . MaxValue )
assertEquals ( List ( 0 , 1 ) , offsetsInLog ( log ) )
assertEquals ( List ( 0 , 1 ) , lastOffsetsPerBatchInLog ( log ) )
}
@Test
def testAbortedTransactionSpanningSegments ( ) : Unit = {
val tp = new TopicPartition ( "test" , 0 )
val cleaner = makeCleaner ( Int . MaxValue )
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , 128 : java . lang . Integer )
val log = makeLog ( config = LogConfig . fromProps ( logConfig . originals , logProps ) )
val producerEpoch = 0. toShort
val producerId = 1L
val appendTransaction = appendTransactionalAsLeader ( log , producerId , producerEpoch )
appendTransaction ( Seq ( 1 ) )
log . roll ( )
log . appendAsLeader ( abortMarker ( producerId , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
log . roll ( )
// Both the batch and the marker should remain after cleaning . The batch is retained
// because it is the last entry for this producerId . The marker is retained because
// there are still batches remaining from this transaction .
cleaner . doClean ( LogToClean ( tp , log , 0L , 100L ) , deleteHorizonMs = Long . MaxValue )
assertEquals ( List ( 1 ) , offsetsInLog ( log ) )
assertEquals ( List ( 0 , 1 ) , lastOffsetsPerBatchInLog ( log ) )
// The empty batch and the marker is still retained after a second cleaning .
cleaner . doClean ( LogToClean ( tp , log , 0L , 100L ) , deleteHorizonMs = Long . MaxValue )
assertEquals ( List ( 1 ) , offsetsInLog ( log ) )
assertEquals ( List ( 0 , 1 ) , lastOffsetsPerBatchInLog ( log ) )
}
@Test
@Test
def testAbortMarkerRemoval ( ) : Unit = {
def testAbortMarkerRemoval ( ) : Unit = {
val tp = new TopicPartition ( "test" , 0 )
val tp = new TopicPartition ( "test" , 0 )
@ -650,7 +703,7 @@ class LogCleanerTest {
// clean the log
// clean the log
val stats = new CleanerStats ( )
val stats = new CleanerStats ( )
cleaner . cleanSegments ( log , Seq ( log . logSegments . head ) , map , 0L , stats )
cleaner . cleanSegments ( log , Seq ( log . logSegments . head ) , map , 0L , stats , new CleanedTransactionMetadata )
val shouldRemain = LogTest . keysInLog ( log ) . filter ( ! keys . contains ( _ ) )
val shouldRemain = LogTest . keysInLog ( log ) . filter ( ! keys . contains ( _ ) )
assertEquals ( shouldRemain , LogTest . keysInLog ( log ) )
assertEquals ( shouldRemain , LogTest . keysInLog ( log ) )
}
}
@ -663,7 +716,7 @@ class LogCleanerTest {
val ( log , offsetMap ) = createLogWithMessagesLargerThanMaxSize ( largeMessageSize = 1024 * 1024 )
val ( log , offsetMap ) = createLogWithMessagesLargerThanMaxSize ( largeMessageSize = 1024 * 1024 )
val cleaner = makeCleaner ( Int . MaxValue , maxMessageSize = 1024 )
val cleaner = makeCleaner ( Int . MaxValue , maxMessageSize = 1024 )
cleaner . cleanSegments ( log , Seq ( log . logSegments . head ) , offsetMap , 0L , new CleanerStats )
cleaner . cleanSegments ( log , Seq ( log . logSegments . head ) , offsetMap , 0L , new CleanerStats , new CleanedTransactionMetadata )
val shouldRemain = LogTest . keysInLog ( log ) . filter ( k => ! offsetMap . map . containsKey ( k . toString ) )
val shouldRemain = LogTest . keysInLog ( log ) . filter ( k => ! offsetMap . map . containsKey ( k . toString ) )
assertEquals ( shouldRemain , LogTest . keysInLog ( log ) )
assertEquals ( shouldRemain , LogTest . keysInLog ( log ) )
}
}
@ -682,7 +735,7 @@ class LogCleanerTest {
val cleaner = makeCleaner ( Int . MaxValue , maxMessageSize = 1024 )
val cleaner = makeCleaner ( Int . MaxValue , maxMessageSize = 1024 )
intercept [ CorruptRecordException ] {
intercept [ CorruptRecordException ] {
cleaner . cleanSegments ( log , Seq ( log . logSegments . head ) , offsetMap , 0L , new CleanerStats )
cleaner . cleanSegments ( log , Seq ( log . logSegments . head ) , offsetMap , 0L , new CleanerStats , new CleanedTransactionMetadata )
}
}
}
}
@ -699,7 +752,7 @@ class LogCleanerTest {
val cleaner = makeCleaner ( Int . MaxValue , maxMessageSize = 1024 )
val cleaner = makeCleaner ( Int . MaxValue , maxMessageSize = 1024 )
intercept [ CorruptRecordException ] {
intercept [ CorruptRecordException ] {
cleaner . cleanSegments ( log , Seq ( log . logSegments . head ) , offsetMap , 0L , new CleanerStats )
cleaner . cleanSegments ( log , Seq ( log . logSegments . head ) , offsetMap , 0L , new CleanerStats , new CleanedTransactionMetadata )
}
}
}
}
@ -1030,7 +1083,8 @@ class LogCleanerTest {
val map = new FakeOffsetMap ( Int . MaxValue )
val map = new FakeOffsetMap ( Int . MaxValue )
keys . foreach ( k => map . put ( key ( k ) , Long . MaxValue ) )
keys . foreach ( k => map . put ( key ( k ) , Long . MaxValue ) )
intercept [ LogCleaningAbortedException ] {
intercept [ LogCleaningAbortedException ] {
cleaner . cleanSegments ( log , log . logSegments . take ( 3 ) . toSeq , map , 0L , new CleanerStats ( ) )
cleaner . cleanSegments ( log , log . logSegments . take ( 3 ) . toSeq , map , 0L , new CleanerStats ( ) ,
new CleanedTransactionMetadata )
}
}
}
}
@ -1239,7 +1293,8 @@ class LogCleanerTest {
// Try to clean segment with offset overflow . This will trigger log split and the cleaning itself must abort .
// Try to clean segment with offset overflow . This will trigger log split and the cleaning itself must abort .
assertThrows [ LogCleaningAbortedException ] {
assertThrows [ LogCleaningAbortedException ] {
cleaner . cleanSegments ( log , List ( segmentWithOverflow ) , offsetMap , 0L , new CleanerStats ( ) )
cleaner . cleanSegments ( log , List ( segmentWithOverflow ) , offsetMap , 0L , new CleanerStats ( ) ,
new CleanedTransactionMetadata )
}
}
assertEquals ( numSegmentsInitial + 1 , log . logSegments . size )
assertEquals ( numSegmentsInitial + 1 , log . logSegments . size )
assertEquals ( allKeys , LogTest . keysInLog ( log ) )
assertEquals ( allKeys , LogTest . keysInLog ( log ) )
@ -1247,7 +1302,8 @@ class LogCleanerTest {
// Clean each segment now that split is complete .
// Clean each segment now that split is complete .
for ( segmentToClean <- log . logSegments )
for ( segmentToClean <- log . logSegments )
cleaner . cleanSegments ( log , List ( segmentToClean ) , offsetMap , 0L , new CleanerStats ( ) )
cleaner . cleanSegments ( log , List ( segmentToClean ) , offsetMap , 0L , new CleanerStats ( ) ,
new CleanedTransactionMetadata )
assertEquals ( expectedKeysAfterCleaning , LogTest . keysInLog ( log ) )
assertEquals ( expectedKeysAfterCleaning , LogTest . keysInLog ( log ) )
assertFalse ( LogTest . hasOffsetOverflow ( log ) )
assertFalse ( LogTest . hasOffsetOverflow ( log ) )
log . close ( )
log . close ( )
@ -1287,7 +1343,8 @@ class LogCleanerTest {
offsetMap . put ( key ( k ) , Long . MaxValue )
offsetMap . put ( key ( k ) , Long . MaxValue )
// clean the log
// clean the log
cleaner . cleanSegments ( log , log . logSegments . take ( 9 ) . toSeq , offsetMap , 0L , new CleanerStats ( ) )
cleaner . cleanSegments ( log , log . logSegments . take ( 9 ) . toSeq , offsetMap , 0L , new CleanerStats ( ) ,
new CleanedTransactionMetadata )
// clear scheduler so that async deletes don 't run
// clear scheduler so that async deletes don 't run
time . scheduler . clear ( )
time . scheduler . clear ( )
var cleanedKeys = LogTest . keysInLog ( log )
var cleanedKeys = LogTest . keysInLog ( log )
@ -1302,7 +1359,8 @@ class LogCleanerTest {
log = recoverAndCheck ( config , allKeys )
log = recoverAndCheck ( config , allKeys )
// clean again
// clean again
cleaner . cleanSegments ( log , log . logSegments . take ( 9 ) . toSeq , offsetMap , 0L , new CleanerStats ( ) )
cleaner . cleanSegments ( log , log . logSegments . take ( 9 ) . toSeq , offsetMap , 0L , new CleanerStats ( ) ,
new CleanedTransactionMetadata )
// clear scheduler so that async deletes don 't run
// clear scheduler so that async deletes don 't run
time . scheduler . clear ( )
time . scheduler . clear ( )
cleanedKeys = LogTest . keysInLog ( log )
cleanedKeys = LogTest . keysInLog ( log )
@ -1323,7 +1381,8 @@ class LogCleanerTest {
}
}
for ( k <- 1 until messageCount by 2 )
for ( k <- 1 until messageCount by 2 )
offsetMap . put ( key ( k ) , Long . MaxValue )
offsetMap . put ( key ( k ) , Long . MaxValue )
cleaner . cleanSegments ( log , log . logSegments . take ( 9 ) . toSeq , offsetMap , 0L , new CleanerStats ( ) )
cleaner . cleanSegments ( log , log . logSegments . take ( 9 ) . toSeq , offsetMap , 0L , new CleanerStats ( ) ,
new CleanedTransactionMetadata )
// clear scheduler so that async deletes don 't run
// clear scheduler so that async deletes don 't run
time . scheduler . clear ( )
time . scheduler . clear ( )
cleanedKeys = LogTest . keysInLog ( log )
cleanedKeys = LogTest . keysInLog ( log )
@ -1340,7 +1399,8 @@ class LogCleanerTest {
}
}
for ( k <- 1 until messageCount by 2 )
for ( k <- 1 until messageCount by 2 )
offsetMap . put ( key ( k ) , Long . MaxValue )
offsetMap . put ( key ( k ) , Long . MaxValue )
cleaner . cleanSegments ( log , log . logSegments . take ( 9 ) . toSeq , offsetMap , 0L , new CleanerStats ( ) )
cleaner . cleanSegments ( log , log . logSegments . take ( 9 ) . toSeq , offsetMap , 0L , new CleanerStats ( ) ,
new CleanedTransactionMetadata )
// clear scheduler so that async deletes don 't run
// clear scheduler so that async deletes don 't run
time . scheduler . clear ( )
time . scheduler . clear ( )
cleanedKeys = LogTest . keysInLog ( log )
cleanedKeys = LogTest . keysInLog ( log )
@ -1568,7 +1628,8 @@ class LogCleanerTest {
appendIdempotentAsLeader ( log , producerId , producerEpoch , isTransactional = true , isFromClient = isFromClient )
appendIdempotentAsLeader ( log , producerId , producerEpoch , isTransactional = true , isFromClient = isFromClient )
}
}
private def appendIdempotentAsLeader ( log : Log , producerId : Long ,
private def appendIdempotentAsLeader ( log : Log ,
producerId : Long ,
producerEpoch : Short ,
producerEpoch : Short ,
isTransactional : Boolean = false ,
isTransactional : Boolean = false ,
leaderEpoch : Int = 0 ,
leaderEpoch : Int = 0 ,