@ -22,6 +22,7 @@ import java.nio.ByteBuffer
import java.nio.file. { Files , Paths }
import java.nio.file. { Files , Paths }
import java.util.Properties
import java.util.Properties
import kafka.api. { ApiVersion , KAFKA_0_11_0_IV0 }
import kafka.common. { OffsetsOutOfOrderException , UnexpectedAppendOffsetException }
import kafka.common. { OffsetsOutOfOrderException , UnexpectedAppendOffsetException }
import kafka.log.Log.DeleteDirSuffix
import kafka.log.Log.DeleteDirSuffix
import kafka.server.epoch. { EpochEntry , LeaderEpochCache , LeaderEpochFileCache }
import kafka.server.epoch. { EpochEntry , LeaderEpochCache , LeaderEpochFileCache }
@ -206,8 +207,6 @@ class LogTest {
// Reload after unclean shutdown with recoveryPoint set to log end offset
// Reload after unclean shutdown with recoveryPoint set to log end offset
log = createLog ( logDir , logConfig , recoveryPoint = logEndOffset )
log = createLog ( logDir , logConfig , recoveryPoint = logEndOffset )
// Note that we don 't maintain the guarantee of having a snapshot for the 2 most recent segments in this case
expectedSnapshotOffsets = Vector ( log . logSegments . last . baseOffset , log . logEndOffset )
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
log . close ( )
log . close ( )
@ -215,15 +214,24 @@ class LogTest {
// Reload after unclean shutdown with recoveryPoint set to 0
// Reload after unclean shutdown with recoveryPoint set to 0
log = createLog ( logDir , logConfig , recoveryPoint = 0L )
log = createLog ( logDir , logConfig , recoveryPoint = 0L )
// Is this working as intended ?
// We progressively create a snapshot for each segment after the recovery point
expectedSnapshotOffsets = log . logSegments . map ( _ . baseOffset ) . tail . toVector : + log.logEndOffset
expectedSnapshotOffsets = log . logSegments . map ( _ . baseOffset ) . tail . toVector : + log.logEndOffset
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
log . close ( )
log . close ( )
}
}
@Test
@Test
def testProducerSnapshotsRecoveryAfterUncleanShutdown ( ) : Unit = {
def testProducerSnapshotsRecoveryAfterUncleanShutdownV1 ( ) : Unit = {
val logConfig = LogTest . createLogConfig ( segmentBytes = 64 * 10 )
testProducerSnapshotsRecoveryAfterUncleanShutdown ( ApiVersion . minSupportedFor ( RecordVersion . V1 ) . version )
}
@Test
def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat ( ) : Unit = {
testProducerSnapshotsRecoveryAfterUncleanShutdown ( ApiVersion . latestVersion . version )
}
private def testProducerSnapshotsRecoveryAfterUncleanShutdown ( messageFormatVersion : String ) : Unit = {
val logConfig = LogTest . createLogConfig ( segmentBytes = 64 * 10 , messageFormatVersion = messageFormatVersion )
var log = createLog ( logDir , logConfig )
var log = createLog ( logDir , logConfig )
assertEquals ( None , log . oldestProducerSnapshotOffset )
assertEquals ( None , log . oldestProducerSnapshotOffset )
@ -247,6 +255,16 @@ class LogTest {
val segmentsWithReads = ArrayBuffer [ LogSegment ] ( )
val segmentsWithReads = ArrayBuffer [ LogSegment ] ( )
val recoveredSegments = ArrayBuffer [ LogSegment ] ( )
val recoveredSegments = ArrayBuffer [ LogSegment ] ( )
val expectedSegmentsWithReads = ArrayBuffer [ Long ] ( )
val expectedSnapshotOffsets = ArrayBuffer [ Long ] ( )
if ( logConfig . messageFormatVersion < KAFKA_0_11_0_IV0 ) {
expectedSegmentsWithReads += activeSegmentOffset
expectedSnapshotOffsets ++= log . logSegments . map ( _ . baseOffset ) . toVector . takeRight ( 2 ) : + log.logEndOffset
} else {
expectedSegmentsWithReads ++= segOffsetsBeforeRecovery ++ Seq ( activeSegmentOffset )
expectedSnapshotOffsets ++= log . logSegments . map ( _ . baseOffset ) . toVector . takeRight ( 4 ) : + log.logEndOffset
}
def createLogWithInterceptedReads ( recoveryPoint : Long ) = {
def createLogWithInterceptedReads ( recoveryPoint : Long ) = {
val maxProducerIdExpirationMs = 60 * 60 * 1000
val maxProducerIdExpirationMs = 60 * 60 * 1000
@ -283,9 +301,8 @@ class LogTest {
ProducerStateManager . deleteSnapshotsBefore ( logDir , segmentOffsets ( segmentOffsets . size - 2 ) )
ProducerStateManager . deleteSnapshotsBefore ( logDir , segmentOffsets ( segmentOffsets . size - 2 ) )
log = createLogWithInterceptedReads ( offsetForRecoveryPointSegment )
log = createLogWithInterceptedReads ( offsetForRecoveryPointSegment )
// We will reload all segments because the recovery point is behind the producer snapshot files ( pre KAFKA - 5829 behaviour )
// We will reload all segments because the recovery point is behind the producer snapshot files ( pre KAFKA - 5829 behaviour )
assertEquals ( segOffsetsBeforeRecovery , segmentsWithReads . map ( _ . baseOffset ) -- Seq ( activeSegment Offset ) )
assertEquals ( expectedSegmentsWithReads , segmentsWithReads . map ( _ . baseOffset ) )
assertEquals ( segOffsetsAfterRecovery , recoveredSegments . map ( _ . baseOffset ) )
assertEquals ( segOffsetsAfterRecovery , recoveredSegments . map ( _ . baseOffset ) )
var expectedSnapshotOffsets = segmentOffsets . takeRight ( 4 ) : + log.logEndOffset
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
log . close ( )
log . close ( )
segmentsWithReads . clear ( )
segmentsWithReads . clear ( )
@ -297,13 +314,12 @@ class LogTest {
log = createLogWithInterceptedReads ( recoveryPoint = recoveryPoint )
log = createLogWithInterceptedReads ( recoveryPoint = recoveryPoint )
assertEquals ( Seq ( activeSegmentOffset ) , segmentsWithReads . map ( _ . baseOffset ) )
assertEquals ( Seq ( activeSegmentOffset ) , segmentsWithReads . map ( _ . baseOffset ) )
assertEquals ( segOffsetsAfterRecovery , recoveredSegments . map ( _ . baseOffset ) )
assertEquals ( segOffsetsAfterRecovery , recoveredSegments . map ( _ . baseOffset ) )
expectedSnapshotOffsets = log . logSegments . map ( _ . baseOffset ) . toVector . takeRight ( 4 ) : + log.logEndOffset
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
// Verify that we keep 2 snapshot files if we checkpoint the log end offset
// Verify that we keep 2 snapshot files if we checkpoint the log end offset
log . deleteSnapshotsAfterRecoveryPointCheckpoint ( )
log . deleteSnapshotsAfterRecoveryPointCheckpoint ( )
expectedSnapshotOffsets = log . logSegments . map ( _ . baseOffset ) . toVector . takeRight ( 2 ) : + log.logEndOffset
val expectedSnapshotsAfterDelete = log . logSegments . map ( _ . baseOffset ) . toVector . takeRight ( 2 ) : + log.logEndOffset
assertEquals ( expectedSnapshotOffsets , listProducerSnapshotOffsets )
assertEquals ( expectedSnapshotsAfterDelete , listProducerSnapshotOffsets )
log . close ( )
log . close ( )
}
}
@ -398,6 +414,9 @@ class LogTest {
// We skip directly to updating the map end offset
// We skip directly to updating the map end offset
stateManager . updateMapEndOffset ( 1L )
stateManager . updateMapEndOffset ( 1L )
EasyMock . expectLastCall ( )
EasyMock . expectLastCall ( )
// Finally , we take a snapshot
stateManager . takeSnapshot ( )
EasyMock . expectLastCall ( ) . once ( )
EasyMock . replay ( stateManager )
EasyMock . replay ( stateManager )
@ -410,14 +429,18 @@ class LogTest {
def testSkipTruncateAndReloadIfOldMessageFormatAndNoCleanShutdown ( ) : Unit = {
def testSkipTruncateAndReloadIfOldMessageFormatAndNoCleanShutdown ( ) : Unit = {
val stateManager = EasyMock . mock ( classOf [ ProducerStateManager ] )
val stateManager = EasyMock . mock ( classOf [ ProducerStateManager ] )
EasyMock . expect ( stateManager . latestSnapshotOffset ) . andReturn ( None )
stateManager . updateMapEndOffset ( 0L )
stateManager . updateMapEndOffset ( 0L )
EasyMock . expectLastCall ( ) . anyTimes ( )
EasyMock . expectLastCall ( ) . anyTimes ( )
stateManager . takeSnapshot ( )
stateManager . takeSnapshot ( )
EasyMock . expectLastCall ( ) . anyTimes ( )
EasyMock . expectLastCall ( ) . anyTimes ( )
EasyMock . expect ( stateManager . isEmpty ) . andReturn ( true )
EasyMock . expectLastCall ( ) . once ( )
EasyMock . expect ( stateManager . firstUnstableOffset ) . andReturn ( None )
EasyMock . expectLastCall ( ) . once ( )
EasyMock . replay ( stateManager )
EasyMock . replay ( stateManager )
val logProps = new Properties ( )
val logProps = new Properties ( )
@ -443,14 +466,18 @@ class LogTest {
def testSkipTruncateAndReloadIfOldMessageFormatAndCleanShutdown ( ) : Unit = {
def testSkipTruncateAndReloadIfOldMessageFormatAndCleanShutdown ( ) : Unit = {
val stateManager = EasyMock . mock ( classOf [ ProducerStateManager ] )
val stateManager = EasyMock . mock ( classOf [ ProducerStateManager ] )
EasyMock . expect ( stateManager . latestSnapshotOffset ) . andReturn ( None )
stateManager . updateMapEndOffset ( 0L )
stateManager . updateMapEndOffset ( 0L )
EasyMock . expectLastCall ( ) . anyTimes ( )
EasyMock . expectLastCall ( ) . anyTimes ( )
stateManager . takeSnapshot ( )
stateManager . takeSnapshot ( )
EasyMock . expectLastCall ( ) . anyTimes ( )
EasyMock . expectLastCall ( ) . anyTimes ( )
EasyMock . expect ( stateManager . isEmpty ) . andReturn ( true )
EasyMock . expectLastCall ( ) . once ( )
EasyMock . expect ( stateManager . firstUnstableOffset ) . andReturn ( None )
EasyMock . expectLastCall ( ) . once ( )
EasyMock . replay ( stateManager )
EasyMock . replay ( stateManager )
val cleanShutdownFile = createCleanShutdownFile ( )
val cleanShutdownFile = createCleanShutdownFile ( )
@ -487,6 +514,12 @@ class LogTest {
stateManager . takeSnapshot ( )
stateManager . takeSnapshot ( )
EasyMock . expectLastCall ( ) . anyTimes ( )
EasyMock . expectLastCall ( ) . anyTimes ( )
EasyMock . expect ( stateManager . isEmpty ) . andReturn ( true )
EasyMock . expectLastCall ( ) . once ( )
EasyMock . expect ( stateManager . firstUnstableOffset ) . andReturn ( None )
EasyMock . expectLastCall ( ) . once ( )
EasyMock . replay ( stateManager )
EasyMock . replay ( stateManager )
val cleanShutdownFile = createCleanShutdownFile ( )
val cleanShutdownFile = createCleanShutdownFile ( )
@ -644,8 +677,12 @@ class LogTest {
assertEquals ( 2 , log . latestProducerStateEndOffset )
assertEquals ( 2 , log . latestProducerStateEndOffset )
log . truncateTo ( 1 )
log . truncateTo ( 1 )
assertEquals ( None , log . latestProducerSnapshotOffset )
assertEquals ( Some ( 1 ) , log . latestProducerSnapshotOffset )
assertEquals ( 1 , log . latestProducerStateEndOffset )
assertEquals ( 1 , log . latestProducerStateEndOffset )
log . truncateTo ( 0 )
assertEquals ( None , log . latestProducerSnapshotOffset )
assertEquals ( 0 , log . latestProducerStateEndOffset )
}
}
@Test
@Test