@ -87,6 +87,159 @@ class LogCleanerTest extends JUnitSuite {
@@ -87,6 +87,159 @@ class LogCleanerTest extends JUnitSuite {
assertEquals ( expectedBytesRead , stats . bytesRead )
}
@Test
def testBasicTransactionAwareCleaning ( ) : Unit = {
val cleaner = makeCleaner ( Int . MaxValue )
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , 2048 : java . lang . Integer )
val log = makeLog ( config = LogConfig . fromProps ( logConfig . originals , logProps ) )
val producerEpoch = 0. toShort
val pid1 = 1
val pid2 = 2
val appendProducer1 = appendTransactionalAsLeader ( log , pid1 , producerEpoch )
val appendProducer2 = appendTransactionalAsLeader ( log , pid2 , producerEpoch )
appendProducer1 ( Seq ( 1 , 2 ) )
appendProducer2 ( Seq ( 2 , 3 ) )
appendProducer1 ( Seq ( 3 , 4 ) )
log . appendAsLeader ( abortMarker ( pid1 , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
log . appendAsLeader ( commitMarker ( pid2 , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
appendProducer1 ( Seq ( 2 ) )
log . appendAsLeader ( commitMarker ( pid1 , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
val abortedTransactions = log . collectAbortedTransactions ( log . logStartOffset , log . logEndOffset )
log . roll ( )
cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , 0L , log . activeSegment . baseOffset ) )
assertEquals ( List ( 3 , 2 ) , keysInLog ( log ) )
assertEquals ( List ( 3 , 6 , 7 , 8 , 9 ) , offsetsInLog ( log ) )
// ensure the transaction index is still correct
assertEquals ( abortedTransactions , log . collectAbortedTransactions ( log . logStartOffset , log . logEndOffset ) )
}
@Test
def testCleanWithTransactionsSpanningSegments ( ) : Unit = {
val cleaner = makeCleaner ( Int . MaxValue )
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , 1024 : java . lang . Integer )
val log = makeLog ( config = LogConfig . fromProps ( logConfig . originals , logProps ) )
val producerEpoch = 0. toShort
val pid1 = 1
val pid2 = 2
val pid3 = 3
val appendProducer1 = appendTransactionalAsLeader ( log , pid1 , producerEpoch )
val appendProducer2 = appendTransactionalAsLeader ( log , pid2 , producerEpoch )
val appendProducer3 = appendTransactionalAsLeader ( log , pid3 , producerEpoch )
appendProducer1 ( Seq ( 1 , 2 ) )
appendProducer3 ( Seq ( 2 , 3 ) )
appendProducer2 ( Seq ( 3 , 4 ) )
log . roll ( )
appendProducer2 ( Seq ( 5 , 6 ) )
appendProducer3 ( Seq ( 6 , 7 ) )
appendProducer1 ( Seq ( 7 , 8 ) )
log . appendAsLeader ( abortMarker ( pid2 , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
appendProducer3 ( Seq ( 8 , 9 ) )
log . appendAsLeader ( commitMarker ( pid3 , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
appendProducer1 ( Seq ( 9 , 10 ) )
log . appendAsLeader ( abortMarker ( pid1 , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
// we have only cleaned the records in the first segment
val dirtyOffset = cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , 0L , log . activeSegment . baseOffset ) ) . _1
assertEquals ( List ( 2 , 3 , 5 , 6 , 6 , 7 , 7 , 8 , 8 , 9 , 9 , 10 ) , keysInLog ( log ) )
log . roll ( )
// append a couple extra segments in the new segment to ensure we have sequence numbers
appendProducer2 ( Seq ( 11 ) )
appendProducer1 ( Seq ( 12 ) )
// finally only the keys from pid3 should remain
cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , dirtyOffset , log . activeSegment . baseOffset ) )
assertEquals ( List ( 2 , 3 , 6 , 7 , 8 , 9 , 11 , 12 ) , keysInLog ( log ) )
}
@Test
def testCommitMarkerRemoval ( ) : Unit = {
val tp = new TopicPartition ( "test" , 0 )
val cleaner = makeCleaner ( Int . MaxValue )
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , 256 : java . lang . Integer )
val log = makeLog ( config = LogConfig . fromProps ( logConfig . originals , logProps ) )
val producerEpoch = 0. toShort
val producerId = 1L
val appendProducer = appendTransactionalAsLeader ( log , producerId , producerEpoch )
appendProducer ( Seq ( 1 ) )
appendProducer ( Seq ( 2 , 3 ) )
log . appendAsLeader ( commitMarker ( producerId , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
appendProducer ( Seq ( 2 ) )
log . appendAsLeader ( commitMarker ( producerId , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
log . roll ( )
// cannot remove the marker in this pass because there are still valid records
var dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , 0L , 100L ) , deleteHorizonMs = time . milliseconds ( ) ) . _1
assertEquals ( List ( 1 , 3 , 2 ) , keysInLog ( log ) )
assertEquals ( List ( 0 , 2 , 3 , 4 , 5 ) , offsetsInLog ( log ) )
appendProducer ( Seq ( 1 , 3 ) )
log . appendAsLeader ( commitMarker ( producerId , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
log . roll ( )
// the first cleaning preserves the commit marker ( at offset 3 ) since there were still records for the transaction
dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = time . milliseconds ( ) ) . _1
assertEquals ( List ( 2 , 1 , 3 ) , keysInLog ( log ) )
assertEquals ( List ( 3 , 4 , 5 , 6 , 7 , 8 ) , offsetsInLog ( log ) )
// delete horizon forced to 0 to verify marker is not removed early
dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = 0L ) . _1
assertEquals ( List ( 2 , 1 , 3 ) , keysInLog ( log ) )
assertEquals ( List ( 3 , 4 , 5 , 6 , 7 , 8 ) , offsetsInLog ( log ) )
// clean again with the delete horizon set back to the current time and verify the marker is removed
cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = time . milliseconds ( ) )
assertEquals ( List ( 2 , 1 , 3 ) , keysInLog ( log ) )
assertEquals ( List ( 4 , 5 , 6 , 7 , 8 ) , offsetsInLog ( log ) )
}
@Test
def testAbortMarkerRemoval ( ) : Unit = {
val tp = new TopicPartition ( "test" , 0 )
val cleaner = makeCleaner ( Int . MaxValue )
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , 256 : java . lang . Integer )
val log = makeLog ( config = LogConfig . fromProps ( logConfig . originals , logProps ) )
val producerEpoch = 0. toShort
val producerId = 1L
val appendProducer = appendTransactionalAsLeader ( log , producerId , producerEpoch )
appendProducer ( Seq ( 1 ) )
appendProducer ( Seq ( 2 , 3 ) )
log . appendAsLeader ( abortMarker ( producerId , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
appendProducer ( Seq ( 3 ) )
log . appendAsLeader ( commitMarker ( producerId , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
log . roll ( )
// delete horizon set to 0 to verify marker is not removed early
val dirtyOffset = cleaner . doClean ( LogToClean ( tp , log , 0L , 100L ) , deleteHorizonMs = 0L ) . _1
assertEquals ( List ( 3 ) , keysInLog ( log ) )
assertEquals ( List ( 3 , 4 , 5 ) , offsetsInLog ( log ) )
// clean again with the delete horizon set back to the current time and verify the marker is removed
cleaner . doClean ( LogToClean ( tp , log , dirtyOffset , 100L ) , deleteHorizonMs = time . milliseconds ( ) )
assertEquals ( List ( 3 ) , keysInLog ( log ) )
assertEquals ( List ( 4 , 5 ) , offsetsInLog ( log ) )
}
/* *
* Test log cleaning with logs containing messages larger than default message size
*/
@ -174,24 +327,45 @@ class LogCleanerTest extends JUnitSuite {
@@ -174,24 +327,45 @@ class LogCleanerTest extends JUnitSuite {
}
@Test
def testLogCleanerRetainsLastWrittenRecordForEachPid ( ) : Unit = {
def testLogCleanerRetainsProducerLastSequence ( ) : Unit = {
val cleaner = makeCleaner ( 10 )
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , 1024 : java . lang . Integer )
val log = makeLog ( config = LogConfig . fromProps ( logConfig . originals , logProps ) )
log . appendAsLeader ( record ( 0 , 0 ) , leaderEpoch = 0 ) // offset 0
log . appendAsLeader ( record ( 0 , 1 , pi d = 1 , epoch = 0 , sequence = 0 ) , leaderEpoch = 0 ) // offset 1
log . appendAsLeader ( record ( 0 , 2 , pi d = 2 , epoch = 0 , sequence = 0 ) , leaderEpoch = 0 ) // offset 2
log . appendAsLeader ( record ( 0 , 3 , pi d = 3 , epoch = 0 , sequence = 0 ) , leaderEpoch = 0 ) // offset 3
log . appendAsLeader ( record ( 1 , 1 , pi d = 2 , epoch = 0 , sequence = 1 ) , leaderEpoch = 0 ) // offset 4
log . appendAsLeader ( record ( 0 , 1 , producerI d = 1 , produc erE poch = 0 , sequence = 0 ) , leaderEpoch = 0 ) // offset 1
log . appendAsLeader ( record ( 0 , 2 , producerI d = 2 , produc erE poch = 0 , sequence = 0 ) , leaderEpoch = 0 ) // offset 2
log . appendAsLeader ( record ( 0 , 3 , producerI d = 3 , produc erE poch = 0 , sequence = 0 ) , leaderEpoch = 0 ) // offset 3
log . appendAsLeader ( record ( 1 , 1 , producerI d = 2 , produc erE poch = 0 , sequence = 1 ) , leaderEpoch = 0 ) // offset 4
// roll the segment , so we can clean the messages already appended
log . roll ( )
cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , 2 , log . activeSegment . baseOffset ) )
assertEquals ( immutable . List ( 0 , 0 , 1 ) , keysInLog ( log ) )
assertEquals ( immutable . List ( 1 , 3 , 4 ) , offsetsInLog ( log ) )
cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , 0L , log . activeSegment . baseOffset ) )
assertEquals ( List ( 0 , 0 , 1 ) , keysInLog ( log ) )
assertEquals ( List ( 1 , 3 , 4 ) , offsetsInLog ( log ) )
}
@Test
def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted ( ) : Unit = {
val cleaner = makeCleaner ( 10 )
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , 1024 : java . lang . Integer )
val log = makeLog ( config = LogConfig . fromProps ( logConfig . originals , logProps ) )
val producerEpoch = 0. toShort
val producerId = 1L
val appendProducer = appendTransactionalAsLeader ( log , producerId , producerEpoch )
appendProducer ( Seq ( 1 ) )
appendProducer ( Seq ( 2 , 3 ) )
log . appendAsLeader ( abortMarker ( producerId , producerEpoch ) , leaderEpoch = 0 , isFromClient = false )
log . roll ( )
cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , 0L , log . activeSegment . baseOffset ) )
assertEquals ( List ( 3 ) , keysInLog ( log ) )
assertEquals ( List ( 2 , 3 ) , offsetsInLog ( log ) )
}
@Test
@ -213,17 +387,17 @@ class LogCleanerTest extends JUnitSuite {
@@ -213,17 +387,17 @@ class LogCleanerTest extends JUnitSuite {
// clean the log with only one message removed
cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , 2 , log . activeSegment . baseOffset ) )
assertEquals ( immutable . List ( 1 , 0 , 1 , 0 ) , keysInLog ( log ) )
assertEquals ( immutable . List ( 1 , 2 , 3 , 4 ) , offsetsInLog ( log ) )
assertEquals ( List ( 1 , 0 , 1 , 0 ) , keysInLog ( log ) )
assertEquals ( List ( 1 , 2 , 3 , 4 ) , offsetsInLog ( log ) )
// continue to make progress , even though we can only clean one message at a time
cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , 3 , log . activeSegment . baseOffset ) )
assertEquals ( immutable . List ( 0 , 1 , 0 ) , keysInLog ( log ) )
assertEquals ( immutable . List ( 2 , 3 , 4 ) , offsetsInLog ( log ) )
assertEquals ( List ( 0 , 1 , 0 ) , keysInLog ( log ) )
assertEquals ( List ( 2 , 3 , 4 ) , offsetsInLog ( log ) )
cleaner . clean ( LogToClean ( new TopicPartition ( "test" , 0 ) , log , 4 , log . activeSegment . baseOffset ) )
assertEquals ( immutable . List ( 1 , 0 ) , keysInLog ( log ) )
assertEquals ( immutable . List ( 3 , 4 ) , offsetsInLog ( log ) )
assertEquals ( List ( 1 , 0 ) , keysInLog ( log ) )
assertEquals ( List ( 3 , 4 ) , offsetsInLog ( log ) )
}
@Test
@ -346,8 +520,12 @@ class LogCleanerTest extends JUnitSuite {
@@ -346,8 +520,12 @@ class LogCleanerTest extends JUnitSuite {
}
/* extract all the keys from a log */
def keysInLog ( log : Log ) : Iterable [ Int ] =
log . logSegments . flatMap ( s => s . log . records . asScala . filter ( _ . hasValue ) . filter ( _ . hasKey ) . map ( record => TestUtils . readString ( record . key ) . toInt ) )
def keysInLog ( log : Log ) : Iterable [ Int ] = {
for ( segment <- log . logSegments ;
batch <- segment . log . batches . asScala if ! batch . isControlBatch ;
record <- batch . asScala if record . hasValue && record . hasKey )
yield TestUtils . readString ( record . key ) . toInt
}
/* extract all the offsets from a log */
def offsetsInLog ( log : Log ) : Iterable [ Long ] =
@ -795,12 +973,12 @@ class LogCleanerTest extends JUnitSuite {
@@ -795,12 +973,12 @@ class LogCleanerTest extends JUnitSuite {
private def messageWithOffset ( key : Int , value : Int , offset : Long ) : MemoryRecords =
messageWithOffset ( key . toString . getBytes , value . toString . getBytes , offset )
def makeLog ( dir : File = dir , config : LogConfig = logConfig ) =
private def makeLog ( dir : File = dir , config : LogConfig = logConfig ) =
new Log ( dir = dir , config = config , logStartOffset = 0L , recoveryPoint = 0L , scheduler = time . scheduler , time = time )
def noOpCheckDone ( topicPartition : TopicPartition ) { /* do nothing */ }
private def noOpCheckDone ( topicPartition : TopicPartition ) { /* do nothing */ }
def makeCleaner ( capacity : Int , checkDone : TopicPartition => Unit = noOpCheckDone , maxMessageSize : Int = 64 * 1024 ) =
private def makeCleaner ( capacity : Int , checkDone : TopicPartition => Unit = noOpCheckDone , maxMessageSize : Int = 64 * 1024 ) =
new Cleaner ( id = 0 ,
offsetMap = new FakeOffsetMap ( capacity ) ,
ioBufferSize = maxMessageSize ,
@ -810,28 +988,62 @@ class LogCleanerTest extends JUnitSuite {
@@ -810,28 +988,62 @@ class LogCleanerTest extends JUnitSuite {
time = time ,
checkDone = checkDone )
def writeToLog ( log : Log , seq : Iterable [ ( Int , Int ) ] ) : Iterable [ Long ] = {
private def writeToLog ( log : Log , seq : Iterable [ ( Int , Int ) ] ) : Iterable [ Long ] = {
for ( ( key , value ) <- seq )
yield log . appendAsLeader ( record ( key , value ) , leaderEpoch = 0 ) . firstOffset
}
def key ( id : Int ) = ByteBuffer . wrap ( id . toString . getBytes )
private def key ( id : Int ) = ByteBuffer . wrap ( id . toString . getBytes )
def record ( key : Int , value : Int , pid : Long = RecordBatch . NO_PRODUCER_ID , epoch : Short = RecordBatch . NO_PRODUCER_EPOCH ,
private def record ( key : Int , value : Int ,
producerId : Long = RecordBatch . NO_PRODUCER_ID ,
producerEpoch : Short = RecordBatch . NO_PRODUCER_EPOCH ,
sequence : Int = RecordBatch . NO_SEQUENCE ,
partitionLeaderEpoch : Int = RecordBatch . NO_PARTITION_LEADER_EPOCH ) : MemoryRecords = {
MemoryRecords . withIdempotentRecords ( RecordBatch . CURRENT_MAGIC_VALUE , 0L , CompressionType . NONE , pid , e poch , sequence ,
MemoryRecords . withIdempotentRecords ( RecordBatch . CURRENT_MAGIC_VALUE , 0L , CompressionType . NONE , producerId , producerE poch , sequence ,
partitionLeaderEpoch , new SimpleRecord ( key . toString . getBytes , value . toString . getBytes ) )
}
def record ( key : Int , value : Array [ Byte ] ) : MemoryRecords =
private def transactionalRecords ( records : Seq [ SimpleRecord ] ,
producerId : Long ,
producerEpoch : Short ,
sequence : Int ) : MemoryRecords = {
MemoryRecords . withTransactionalRecords ( CompressionType . NONE , producerId , producerEpoch , sequence , records : _ * )
}
private def appendTransactionalAsLeader ( log : Log , producerId : Long , producerEpoch : Short = 0 ) : Seq [ Int ] => Unit = {
var sequence = 0
keys : Seq [ Int ] => {
val simpleRecords = keys . map { key =>
val keyBytes = key . toString . getBytes
new SimpleRecord ( keyBytes , keyBytes ) // the value doesn 't matter too much since we validate offsets
}
val records = transactionalRecords ( simpleRecords , producerId , producerEpoch , sequence )
log . appendAsLeader ( records , leaderEpoch = 0 )
sequence += simpleRecords . size
}
}
private def commitMarker ( producerId : Long , producerEpoch : Short , timestamp : Long = time . milliseconds ( ) ) : MemoryRecords =
endTxnMarker ( producerId , producerEpoch , ControlRecordType . COMMIT , 0L , timestamp )
private def abortMarker ( producerId : Long , producerEpoch : Short , timestamp : Long = time . milliseconds ( ) ) : MemoryRecords =
endTxnMarker ( producerId , producerEpoch , ControlRecordType . ABORT , 0L , timestamp )
private def endTxnMarker ( producerId : Long , producerEpoch : Short , controlRecordType : ControlRecordType ,
offset : Long , timestamp : Long ) : MemoryRecords = {
val endTxnMarker = new EndTransactionMarker ( controlRecordType , 0 )
MemoryRecords . withEndTransactionMarker ( offset , timestamp , RecordBatch . NO_PARTITION_LEADER_EPOCH ,
producerId , producerEpoch , endTxnMarker )
}
private def record ( key : Int , value : Array [ Byte ] ) : MemoryRecords =
TestUtils . singletonRecords ( key = key . toString . getBytes , value = value )
def unkeyedRecord ( value : Int ) : MemoryRecords =
private def unkeyedRecord ( value : Int ) : MemoryRecords =
TestUtils . singletonRecords ( value = value . toString . getBytes )
def tombstoneRecord ( key : Int ) : MemoryRecords = record ( key , null )
private def tombstoneRecord ( key : Int ) : MemoryRecords = record ( key , null )
}
@ -842,12 +1054,12 @@ class FakeOffsetMap(val slots: Int) extends OffsetMap {
@@ -842,12 +1054,12 @@ class FakeOffsetMap(val slots: Int) extends OffsetMap {
private def keyFor ( key : ByteBuffer ) =
new String ( Utils . readBytes ( key . duplicate ) , "UTF-8" )
def put ( key : ByteBuffer , offset : Long ) : Unit = {
override def put ( key : ByteBuffer , offset : Long ) : Unit = {
lastOffset = offset
map . put ( keyFor ( key ) , offset )
}
def get ( key : ByteBuffer ) : Long = {
override def get ( key : ByteBuffer ) : Long = {
val k = keyFor ( key )
if ( map . containsKey ( k ) )
map . get ( k )
@ -855,11 +1067,15 @@ class FakeOffsetMap(val slots: Int) extends OffsetMap {
@@ -855,11 +1067,15 @@ class FakeOffsetMap(val slots: Int) extends OffsetMap {
- 1L
}
def clear ( ) : Unit = map . clear ( )
override def clear ( ) : Unit = map . clear ( )
def size : Int = map . size
override def size : Int = map . size
override def latestOffset : Long = lastOffset
def latestOffset : Long = lastOffset
override def updateLatestOffset ( offset : Long ) : Unit = {
lastOffset = offset
}
override def toString : String = map . toString
}