@ -77,6 +77,42 @@ class LogCleanerManagerTest extends Logging {
@@ -77,6 +77,42 @@ class LogCleanerManagerTest extends Logging {
logs
}
@Test
def testGrabFilthiestCompactedLogThrowsException ( ) : Unit = {
val tp = new TopicPartition ( "A" , 1 )
val logSegmentSize = TestUtils . singletonRecords ( "test" . getBytes ) . sizeInBytes * 10
val logSegmentsCount = 2
val tpDir = new File ( logDir , "A-1" )
// the exception should be catched and the partition that caused it marked as uncleanable
class LogMock ( dir : File , config : LogConfig ) extends Log ( dir , config , 0L , 0L ,
time . scheduler , new BrokerTopicStats , time , 60 * 60 * 1000 , LogManager . ProducerIdExpirationCheckIntervalMs ,
topicPartition , new ProducerStateManager ( tp , tpDir , 60 * 60 * 1000 ) , new LogDirFailureChannel ( 10 ) ) {
// Throw an error in getFirstBatchTimestampForSegments since it is called in grabFilthiestLog ( )
override def getFirstBatchTimestampForSegments ( segments : Iterable [ LogSegment ] ) : Iterable [ Long ] =
throw new IllegalStateException ( "Error!" )
}
val log : Log = new LogMock ( tpDir , createLowRetentionLogConfig ( logSegmentSize , LogConfig . Compact ) )
writeRecords ( log = log ,
numBatches = logSegmentsCount * 2 ,
recordsPerBatch = 10 ,
batchesPerSegment = 2
)
val logsPool = new Pool [ TopicPartition , Log ] ( )
logsPool . put ( tp , log )
val cleanerManager = createCleanerManagerMock ( logsPool )
cleanerCheckpoints . put ( tp , 1 )
val thrownException = intercept [ LogCleaningException ] {
cleanerManager . grabFilthiestCompactedLog ( time ) . get
}
assertEquals ( log , thrownException . log )
assertTrue ( thrownException . getCause . isInstanceOf [ IllegalStateException ] )
}
@Test
def testGrabFilthiestCompactedLogReturnsLogWithDirtiestRatio ( ) : Unit = {
val tp0 = new TopicPartition ( "wishing-well" , 0 )
@ -505,13 +541,7 @@ class LogCleanerManagerTest extends Logging {
@@ -505,13 +541,7 @@ class LogCleanerManagerTest extends Logging {
private def createLog ( segmentSize : Int ,
cleanupPolicy : String ,
topicPartition : TopicPartition = new TopicPartition ( "log" , 0 ) ) : Log = {
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , segmentSize : Integer )
logProps . put ( LogConfig . RetentionMsProp , 1 : Integer )
logProps . put ( LogConfig . CleanupPolicyProp , cleanupPolicy )
logProps . put ( LogConfig . MinCleanableDirtyRatioProp , 0.05 : java . lang . Double ) // small for easier and clearer tests
val config = LogConfig ( logProps )
val config = createLowRetentionLogConfig ( segmentSize , cleanupPolicy )
val partitionDir = new File ( logDir , Log . logDirName ( topicPartition ) )
Log ( partitionDir ,
@ -526,6 +556,16 @@ class LogCleanerManagerTest extends Logging {
@@ -526,6 +556,16 @@ class LogCleanerManagerTest extends Logging {
logDirFailureChannel = new LogDirFailureChannel ( 10 ) )
}
private def createLowRetentionLogConfig ( segmentSize : Int , cleanupPolicy : String ) : LogConfig = {
val logProps = new Properties ( )
logProps . put ( LogConfig . SegmentBytesProp , segmentSize : Integer )
logProps . put ( LogConfig . RetentionMsProp , 1 : Integer )
logProps . put ( LogConfig . CleanupPolicyProp , cleanupPolicy )
logProps . put ( LogConfig . MinCleanableDirtyRatioProp , 0.05 : java . lang . Double ) // small for easier and clearer tests
LogConfig ( logProps )
}
private def writeRecords ( log : Log ,
numBatches : Int ,
recordsPerBatch : Int ,