@ -16,6 +16,7 @@
@@ -16,6 +16,7 @@
*/
package kafka.server.epoch
import java.io.File
import kafka.server.LogOffsetMetadata
@ -24,7 +25,7 @@ import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFIN
@@ -24,7 +25,7 @@ import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFIN
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.junit.Assert._
import org.junit. { Before , Test }
import org.junit.Test
import scala.collection.mutable.ListBuffer
@ -33,54 +34,44 @@ import scala.collection.mutable.ListBuffer
@@ -33,54 +34,44 @@ import scala.collection.mutable.ListBuffer
*/
class LeaderEpochFileCacheTest {
val tp = new TopicPartition ( "TestTopic" , 5 )
var checkpoint : LeaderEpochCheckpoint = _
private var logEndOffset = 0L
private val checkpoint : LeaderEpochCheckpoint = new LeaderEpochCheckpoint {
private var epochs : Seq [ EpochEntry ] = Seq ( )
override def write ( epochs : Seq [ EpochEntry ] ) : Unit = this . epochs = epochs
override def read ( ) : Seq [ EpochEntry ] = this . epochs
}
private val cache = new LeaderEpochFileCache ( tp , logEndOffset _ , checkpoint )
@Test
def shouldAddEpochAndMessageOffsetToCache ( ) = {
var leo = 0
def leoFinder ( ) = new LogOffsetMetadata ( leo )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// When
cache . assign ( epoch = 2 , o ffset = 10 )
leo = 11
cache . assign ( epoch = 2 , startOffset = 10 )
logEndOffset = 11
// Then
assertEquals ( 2 , cache . latestEpoch ( ) )
assertEquals ( EpochEntry ( 2 , 10 ) , cache . epochEntries ( ) ( 0 ) )
assertEquals ( ( 2 , leo ) , cache . endOffsetFor ( 2 ) ) // should match leo
assertEquals ( 2 , cache . latestEpoch )
assertEquals ( EpochEntry ( 2 , 10 ) , cache . epochEntries ( 0 ) )
assertEquals ( ( 2 , logEndOffset ) , cache . endOffsetFor ( 2 ) ) // should match logEndOffset
}
@Test
def shouldReturnLogEndOffsetIfLatestEpochRequested ( ) = {
var leo = 0
def leoFinder ( ) = new LogOffsetMetadata ( leo )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// When just one epoch
cache . assign ( epoch = 2 , o ffset = 11 )
cache . assign ( epoch = 2 , o ffset = 12 )
leo = 14
cache . assign ( epoch = 2 , startOffset = 11 )
cache . assign ( epoch = 2 , startOffset = 12 )
logEndOffset = 14
// Then
assertEquals ( ( 2 , leo ) , cache . endOffsetFor ( 2 ) )
assertEquals ( ( 2 , logEndOffset ) , cache . endOffsetFor ( 2 ) )
}
@Test
def shouldReturnUndefinedOffsetIfUndefinedEpochRequested ( ) = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
val expectedEpochEndOffset = ( UNDEFINED_EPOCH , UNDEFINED_EPOCH_OFFSET )
// Given cache with some data on leader
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// assign couple of epochs
cache . assign ( epoch = 2 , o ffset = 11 )
cache . assign ( epoch = 3 , o ffset = 12 )
cache . assign ( epoch = 2 , startOffset = 11 )
cache . assign ( epoch = 3 , startOffset = 12 )
// When ( say a bootstraping follower ) sends request for UNDEFINED_EPOCH
val epochAndOffsetFor = cache . endOffsetFor ( UNDEFINED_EPOCH )
@ -92,68 +83,51 @@ class LeaderEpochFileCacheTest {
@@ -92,68 +83,51 @@ class LeaderEpochFileCacheTest {
@Test
def shouldNotOverwriteLogEndOffsetForALeaderEpochOnceItHasBeenAssigned ( ) = {
var leo = 0
def leoFinder ( ) = new LogOffsetMetadata ( leo )
// Given
leo = 9
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
logEndOffset = 9
cache . assign ( 2 , leo )
cache . assign ( 2 , logEndOffset )
// When called again later
cache . assign ( 2 , 10 )
// Then the offset should NOT have been updated
assertEquals ( leo , cache . epochEntries ( ) ( 0 ) . startOffset )
assertEquals ( logEndOffset , cache . epochEntries ( 0 ) . startOffset )
assertEquals ( ListBuffer ( EpochEntry ( 2 , 9 ) ) , cache . epochEntries )
}
@Test
def shouldAllowLeaderEpochToChangeEvenIfOffsetDoesNot ( ) = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
def shouldEnforceMonotonicallyIncreasingStartOffsets ( ) = {
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( 2 , 9 )
// When update epoch new epoch but same offset
cache . assign ( 3 , 9 )
// Then epoch should have been updated
assertEquals ( ListBuffer ( EpochEntry ( 2 , 9 ) , EpochEntry ( 3 , 9 ) ) , cache . epochEntries ( ) )
assertEquals ( ListBuffer ( EpochEntry ( 3 , 9 ) ) , cache . epochEntries )
}
@Test
def shouldNotOverwriteOffsetForALeaderEpochOnceItHasBeenAssigned ( ) = {
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => new LogOffsetMetadata ( 0 ) , checkpoint )
cache . assign ( 2 , 6 )
// When called again later with a greater offset
cache . assign ( 2 , 10 )
// Then later update should have been ignored
assertEquals ( 6 , cache . epochEntries ( ) ( 0 ) . startOffset )
assertEquals ( 6 , cache . epochEntries ( 0 ) . startOffset )
}
@Test
def shouldReturnUnsupportedIfNoEpochRecorded ( ) {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// Then
assertEquals ( ( UNDEFINED_EPOCH , UNDEFINED_EPOCH_OFFSET ) , cache . endOffsetFor ( 0 ) )
}
@Test
def shouldReturnUnsupportedIfNoEpochRecordedAndUndefinedEpochRequested ( ) {
val leo = 73
def leoFinder ( ) = new LogOffsetMetadata ( leo )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
logEndOffset = 73
// When ( say a follower on older message format version ) sends request for UNDEFINED_EPOCH
val offsetFor = cache . endOffsetFor ( UNDEFINED_EPOCH )
@ -164,39 +138,41 @@ class LeaderEpochFileCacheTest {
@@ -164,39 +138,41 @@ class LeaderEpochFileCacheTest {
}
@Test
def shouldReturnUnsupportedIfRequestedEpochLessThanFirstEpoch ( ) {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 5 , offset = 11 )
cache . assign ( epoch = 6 , offset = 12 )
cache . assign ( epoch = 7 , offset = 13 )
def shouldReturnFirstEpochIfRequestedEpochLessThanFirstEpoch ( ) {
cache . assign ( epoch = 5 , startOffset = 11 )
cache . assign ( epoch = 6 , startOffset = 12 )
cache . assign ( epoch = 7 , startOffset = 13 )
// When
val epochAndOffset = cache . endOffsetFor ( 5 - 1 )
val epochAndOffset = cache . endOffsetFor ( 4 )
// Then
assertEquals ( ( UNDEFINED_EPOCH , UNDEFINED_EPOCH_OFFSET ) , epochAndOffset )
assertEquals ( ( 4 , 11 ) , epochAndOffset )
}
@Test
def shouldGetFirstOffsetOfSubsequentEpochWhenOffsetRequestedForPreviousEpoch ( ) = {
var leo = 0
def leoFinder ( ) = new LogOffsetMetadata ( leo )
def shouldTruncateIfMatchingEpochButEarlierStartingOffset ( ) : Unit = {
cache . assign ( epoch = 5 , startOffset = 11 )
cache . assign ( epoch = 6 , startOffset = 12 )
cache . assign ( epoch = 7 , startOffset = 13 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// epoch 7 starts at an earlier offset
cache . assign ( epoch = 7 , startOffset = 12 )
assertEquals ( ( 5 , 12 ) , cache . endOffsetFor ( 5 ) )
assertEquals ( ( 5 , 12 ) , cache . endOffsetFor ( 6 ) )
}
@Test
def shouldGetFirstOffsetOfSubsequentEpochWhenOffsetRequestedForPreviousEpoch ( ) = {
// When several epochs
cache . assign ( epoch = 1 , offset = 11 )
cache . assign ( epoch = 1 , offset = 12 )
cache . assign ( epoch = 2 , offset = 13 )
cache . assign ( epoch = 2 , offset = 14 )
cache . assign ( epoch = 3 , offset = 15 )
cache . assign ( epoch = 3 , o ffset = 16 )
leo = 17
cache . assign ( epoch = 1 , startO ffset = 11 )
cache . assign ( epoch = 1 , startO ffset = 12 )
cache . assign ( epoch = 2 , startO ffset = 13 )
cache . assign ( epoch = 2 , startO ffset = 14 )
cache . assign ( epoch = 3 , startO ffset = 15 )
cache . assign ( epoch = 3 , startO ffset = 16 )
logEndOffset = 17
// Then get the start offset of the next epoch
assertEquals ( ( 2 , 15 ) , cache . endOffsetFor ( 2 ) )
@ -204,15 +180,10 @@ class LeaderEpochFileCacheTest {
@@ -204,15 +180,10 @@ class LeaderEpochFileCacheTest {
@Test
def shouldReturnNextAvailableEpochIfThereIsNoExactEpochForTheOneRequested ( ) {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// When
cache . assign ( epoch = 0 , o ffset = 10 )
cache . assign ( epoch = 2 , o ffset = 13 )
cache . assign ( epoch = 4 , o ffset = 17 )
cache . assign ( epoch = 0 , startOffset = 10 )
cache . assign ( epoch = 2 , startOffset = 13 )
cache . assign ( epoch = 4 , startOffset = 17 )
// Then
assertEquals ( ( 0 , 13 ) , cache . endOffsetFor ( requestedEpoch = 1 ) )
@ -222,14 +193,9 @@ class LeaderEpochFileCacheTest {
@@ -222,14 +193,9 @@ class LeaderEpochFileCacheTest {
@Test
def shouldNotUpdateEpochAndStartOffsetIfItDidNotChange ( ) = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// When
cache . assign ( epoch = 2 , o ffset = 6 )
cache . assign ( epoch = 2 , o ffset = 7 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 2 , startOffset = 7 )
// Then
assertEquals ( 1 , cache . epochEntries . size )
@ -238,14 +204,10 @@ class LeaderEpochFileCacheTest {
@@ -238,14 +204,10 @@ class LeaderEpochFileCacheTest {
@Test
def shouldReturnInvalidOffsetIfEpochIsRequestedWhichIsNotCurrentlyTracked ( ) : Unit = {
val leo = 100
def leoFinder ( ) = new LogOffsetMetadata ( leo )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
logEndOffset = 100
// When
cache . assign ( epoch = 2 , o ffset = 100 )
cache . assign ( epoch = 2 , startOffset = 100 )
// Then
assertEquals ( ( UNDEFINED_EPOCH , UNDEFINED_EPOCH_OFFSET ) , cache . endOffsetFor ( 3 ) )
@ -253,35 +215,28 @@ class LeaderEpochFileCacheTest {
@@ -253,35 +215,28 @@ class LeaderEpochFileCacheTest {
@Test
def shouldSupportEpochsThatDoNotStartFromZero ( ) : Unit = {
var leo = 0
def leoFinder ( ) = new LogOffsetMetadata ( leo )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// When
cache . assign ( epoch = 2 , o ffset = 6 )
leo = 7
cache . assign ( epoch = 2 , startOffset = 6 )
logEndOffset = 7
// Then
assertEquals ( ( 2 , leo ) , cache . endOffsetFor ( 2 ) )
assertEquals ( ( 2 , logEndOffset ) , cache . endOffsetFor ( 2 ) )
assertEquals ( 1 , cache . epochEntries . size )
assertEquals ( EpochEntry ( 2 , 6 ) , cache . epochEntries ( ) ( 0 ) )
assertEquals ( EpochEntry ( 2 , 6 ) , cache . epochEntries ( 0 ) )
}
@Test
def shouldPersistEpochsBetweenInstances ( ) {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
val checkpointPath = TestUtils . tempFile ( ) . getAbsolutePath
checkpoint = new LeaderEpochCheckpointFile ( new File ( checkpointPath ) )
val checkpoint = new LeaderEpochCheckpointFile ( new File ( checkpointPath ) )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , o ffset = 6 )
val cache = new LeaderEpochFileCache ( tp , logEndOffset _ , checkpoint )
cache . assign ( epoch = 2 , startO ffset = 6 )
// When
val checkpoint2 = new LeaderEpochCheckpointFile ( new File ( checkpointPath ) )
val cache2 = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint2 )
val cache2 = new LeaderEpochFileCache ( tp , logEndOffset _ , checkpoint2 )
// Then
assertEquals ( 1 , cache2 . epochEntries . size )
@ -289,81 +244,68 @@ class LeaderEpochFileCacheTest {
@@ -289,81 +244,68 @@ class LeaderEpochFileCacheTest {
}
@Test
def shouldNotLetEpochGoBackwardsEvenIfMessageEpochsDo ( ) : Unit = {
var leo = 0
def leoFinder ( ) = new LogOffsetMetadata ( leo )
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
def shouldEnforceMonotonicallyIncreasingEpochs ( ) : Unit = {
// Given
cache . assign ( epoch = 1 , o ffset = 5 ) ; leo = 6
cache . assign ( epoch = 2 , o ffset = 6 ) ; leo = 7
cache . assign ( epoch = 1 , startOffset = 5 ) ; logEndOffset = 6
cache . assign ( epoch = 2 , startOffset = 6 ) ; logEndOffset = 7
// When we update an epoch in the past with an earlier offset
cache . assign ( epoch = 1 , offset = 7 ) ; leo = 8
// When we update an epoch in the past with a different offset , the log has already reached
// an inconsistent state . Our options are either to raise an error , ignore the new append ,
// or truncate the cached epochs to the point of conflict . We take this latter approach in
// order to guarantee that epochs and offsets in the cache increase monotonically , which makes
// the search logic simpler to reason about .
cache . assign ( epoch = 1 , startOffset = 7 ) ; logEndOffset = 8
// Then epoch should not be changed
assertEquals ( 2 , cache . latestEpoch ( ) )
// Then later epochs will be remov ed
assertEquals ( 1 , cache . latestEpoch )
// Then end offset for epoch 1 shouldn 't have changed
assertEquals ( ( 1 , 6 ) , cache . endOffsetFor ( 1 ) )
// Then end offset for epoch 2 has to be the offset of the epoch 1 message ( I can 't think of a better option )
assertEquals ( ( 2 , 8 ) , cache . endOffsetFor ( 2 ) )
// Then end offset for epoch 1 will have changed
assertEquals ( ( 1 , 8 ) , cache . endOffsetFor ( 1 ) )
// Epoch history shouldn 't have chang ed
assertEquals ( EpochEntry ( 1 , 5 ) , cache . epochEntries ( ) ( 0 ) )
assertEquals ( EpochEntry ( 2 , 6 ) , cache . epochEntries ( ) ( 1 ) )
// Then end offset for epoch 2 is now undefined
assertEquals ( ( UNDEFINED_EPOCH , UNDEFINED_EPOCH_OFFSET ) , cache . endOffsetFor ( 2 ) )
assertEquals ( EpochEntry ( 1 , 7 ) , cache . epochEntries ( 0 ) )
}
@Test
def shouldNotLetOffsetsGoBackwardsEvenIfEpochsProgress ( ) = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
def shouldEnforceOffsetsIncreaseMonotonically ( ) = {
// When epoch goes forward but offset goes backwards
cache . assign ( epoch = 2 , o ffset = 6 )
cache . assign ( epoch = 3 , o ffset = 5 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 5 )
// Then latter assign should be ignored
assertEquals ( EpochEntry ( 2 , 6 ) , cache . epochEntries . toList ( 0 ) )
// The last assignment wins and the conflicting one is removed from the log
assertEquals ( EpochEntry ( 3 , 5 ) , cache . epochEntries . toList ( 0 ) )
}
@Test
def shouldIncreaseAndTrackEpochsAsLeadersChangeManyTimes ( ) : Unit = {
var leo = 0
def leoFinder ( ) = new LogOffsetMetadata ( leo )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 0 , offset = 0 ) // leo = 0
cache . assign ( epoch = 0 , startOffset = 0 ) // logEndOffset = 0
// When
cache . assign ( epoch = 1 , o ffset = 0 ) // leo = 0
cache . assign ( epoch = 1 , startO ffset = 0 ) // logEndOffset = 0
// Then epoch should go up
assertEquals ( 1 , cache . latestEpoch ( ) )
assertEquals ( 1 , cache . latestEpoch )
// offset for 1 should still be 0
assertEquals ( ( 1 , 0 ) , cache . endOffsetFor ( 1 ) )
// offset for epoch 0 should still be 0
assertEquals ( ( 0 , 0 ) , cache . endOffsetFor ( 0 ) )
// When we write 5 messages as epoch 1
leo = 5
logEndOffset = 5
// Then end offset for epoch ( 1 ) should be leo => 5
// Then end offset for epoch ( 1 ) should be logEndOffset => 5
assertEquals ( ( 1 , 5 ) , cache . endOffsetFor ( 1 ) )
// Epoch 0 should still be at offset 0
assertEquals ( ( 0 , 0 ) , cache . endOffsetFor ( 0 ) )
// When
cache . assign ( epoch = 2 , o ffset = 5 ) // leo = 5
cache . assign ( epoch = 2 , startO ffset = 5 ) // logEndOffset = 5
leo = 10 // write another 5 messages
logEndOffset = 10 // write another 5 messages
// Then end offset for epoch ( 2 ) should be leo => 10
// Then end offset for epoch ( 2 ) should be logEndOffset => 10
assertEquals ( ( 2 , 10 ) , cache . endOffsetFor ( 2 ) )
// end offset for epoch ( 1 ) should be the start offset of epoch ( 2 ) => 5
@ -375,36 +317,30 @@ class LeaderEpochFileCacheTest {
@@ -375,36 +317,30 @@ class LeaderEpochFileCacheTest {
@Test
def shouldIncreaseAndTrackEpochsAsFollowerReceivesManyMessages ( ) : Unit = {
var leo = 0
def leoFinder ( ) = new LogOffsetMetadata ( leo )
// When new
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// When Messages come in
cache . assign ( epoch = 0 , o ffset = 0 ) ; leo = 1
cache . assign ( epoch = 0 , o ffset = 1 ) ; leo = 2
cache . assign ( epoch = 0 , o ffset = 2 ) ; leo = 3
cache . assign ( epoch = 0 , startOffset = 0 ) ; logEndOffset = 1
cache . assign ( epoch = 0 , startOffset = 1 ) ; logEndOffset = 2
cache . assign ( epoch = 0 , startOffset = 2 ) ; logEndOffset = 3
// Then epoch should stay , offsets should grow
assertEquals ( 0 , cache . latestEpoch ( ) )
assertEquals ( ( 0 , leo ) , cache . endOffsetFor ( 0 ) )
assertEquals ( 0 , cache . latestEpoch )
assertEquals ( ( 0 , logEndOffset ) , cache . endOffsetFor ( 0 ) )
// When messages arrive with greater epoch
cache . assign ( epoch = 1 , o ffset = 3 ) ; leo = 4
cache . assign ( epoch = 1 , o ffset = 4 ) ; leo = 5
cache . assign ( epoch = 1 , o ffset = 5 ) ; leo = 6
cache . assign ( epoch = 1 , startO ffset = 3 ) ; logEndOffset = 4
cache . assign ( epoch = 1 , startO ffset = 4 ) ; logEndOffset = 5
cache . assign ( epoch = 1 , startO ffset = 5 ) ; logEndOffset = 6
assertEquals ( 1 , cache . latestEpoch ( ) )
assertEquals ( ( 1 , leo ) , cache . endOffsetFor ( 1 ) )
assertEquals ( 1 , cache . latestEpoch )
assertEquals ( ( 1 , logEndOffset ) , cache . endOffsetFor ( 1 ) )
// When
cache . assign ( epoch = 2 , o ffset = 6 ) ; leo = 7
cache . assign ( epoch = 2 , o ffset = 7 ) ; leo = 8
cache . assign ( epoch = 2 , o ffset = 8 ) ; leo = 9
cache . assign ( epoch = 2 , startO ffset = 6 ) ; logEndOffset = 7
cache . assign ( epoch = 2 , startO ffset = 7 ) ; logEndOffset = 8
cache . assign ( epoch = 2 , startO ffset = 8 ) ; logEndOffset = 9
assertEquals ( 2 , cache . latestEpoch ( ) )
assertEquals ( ( 2 , leo ) , cache . endOffsetFor ( 2 ) )
assertEquals ( 2 , cache . latestEpoch )
assertEquals ( ( 2 , logEndOffset ) , cache . endOffsetFor ( 2 ) )
// Older epochs should return the start offset of the first message in the subsequent epoch .
assertEquals ( ( 0 , 3 ) , cache . endOffsetFor ( 0 ) )
@ -413,16 +349,13 @@ class LeaderEpochFileCacheTest {
@@ -413,16 +349,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldDropEntriesOnEpochBoundaryWhenRemovingLatestEntries ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When clear latest on epoch boundary
cache . clearAndFlushLatest ( o ffset = 8 )
cache . truncateFromEnd ( endO ffset = 8 )
// Then should remove two latest epochs ( remove is inclusive )
assertEquals ( ListBuffer ( EpochEntry ( 2 , 6 ) ) , cache . epochEntries )
@ -430,16 +363,13 @@ class LeaderEpochFileCacheTest {
@@ -430,16 +363,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldPreserveResetOffsetOnClearEarliestIfOneExists ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When reset to offset ON epoch boundary
cache . clearAndFlushEarliest ( o ffset = 8 )
cache . truncateFromStart ( startO ffset = 8 )
// Then should preserve ( 3 , 8 )
assertEquals ( ListBuffer ( EpochEntry ( 3 , 8 ) , EpochEntry ( 4 , 11 ) ) , cache . epochEntries )
@ -447,16 +377,13 @@ class LeaderEpochFileCacheTest {
@@ -447,16 +377,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldUpdateSavedOffsetWhenOffsetToClearToIsBetweenEpochs ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When reset to offset BETWEEN epoch boundaries
cache . clearAndFlushEarliest ( o ffset = 9 )
cache . truncateFromStart ( startO ffset = 9 )
// Then we should retain epoch 3 , but update it 's offset to 9 as 8 has been removed
assertEquals ( ListBuffer ( EpochEntry ( 3 , 9 ) , EpochEntry ( 4 , 11 ) ) , cache . epochEntries )
@ -464,16 +391,13 @@ class LeaderEpochFileCacheTest {
@@ -464,16 +391,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldNotClearAnythingIfOffsetToEarly ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When reset to offset before first epoch offset
cache . clearAndFlushEarliest ( o ffset = 1 )
cache . truncateFromStart ( startO ffset = 1 )
// Then nothing should change
assertEquals ( ListBuffer ( EpochEntry ( 2 , 6 ) , EpochEntry ( 3 , 8 ) , EpochEntry ( 4 , 11 ) ) , cache . epochEntries )
@ -481,16 +405,13 @@ class LeaderEpochFileCacheTest {
@@ -481,16 +405,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldNotClearAnythingIfOffsetToFirstOffset ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When reset to offset on earliest epoch boundary
cache . clearAndFlushEarliest ( o ffset = 6 )
cache . truncateFromStart ( startO ffset = 6 )
// Then nothing should change
assertEquals ( ListBuffer ( EpochEntry ( 2 , 6 ) , EpochEntry ( 3 , 8 ) , EpochEntry ( 4 , 11 ) ) , cache . epochEntries )
@ -498,16 +419,13 @@ class LeaderEpochFileCacheTest {
@@ -498,16 +419,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldRetainLatestEpochOnClearAllEarliest ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When
cache . clearAndFlushEarliest ( o ffset = 11 )
cache . truncateFromStart ( startO ffset = 11 )
// Then retain the last
assertEquals ( ListBuffer ( EpochEntry ( 4 , 11 ) ) , cache . epochEntries )
@ -515,16 +433,13 @@ class LeaderEpochFileCacheTest {
@@ -515,16 +433,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldUpdateOffsetBetweenEpochBoundariesOnClearEarliest ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When we clear from a postition between offset 8 & offset 11
cache . clearAndFlushEarliest ( o ffset = 9 )
cache . truncateFromStart ( startO ffset = 9 )
// Then we should update the middle epoch entry 's offset
assertEquals ( ListBuffer ( EpochEntry ( 3 , 9 ) , EpochEntry ( 4 , 11 ) ) , cache . epochEntries )
@ -532,16 +447,13 @@ class LeaderEpochFileCacheTest {
@@ -532,16 +447,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldUpdateOffsetBetweenEpochBoundariesOnClearEarliest2 ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 0 , offset = 0 )
cache . assign ( epoch = 1 , offset = 7 )
cache . assign ( epoch = 2 , offset = 10 )
cache . assign ( epoch = 0 , startOffset = 0 )
cache . assign ( epoch = 1 , startOffset = 7 )
cache . assign ( epoch = 2 , startOffset = 10 )
// When we clear from a postition between offset 0 & offset 7
cache . clearAndFlushEarliest ( o ffset = 5 )
cache . truncateFromStart ( startO ffset = 5 )
// Then we should keeep epoch 0 but update the offset appropriately
assertEquals ( ListBuffer ( EpochEntry ( 0 , 5 ) , EpochEntry ( 1 , 7 ) , EpochEntry ( 2 , 10 ) ) , cache . epochEntries )
@ -549,16 +461,13 @@ class LeaderEpochFileCacheTest {
@@ -549,16 +461,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldRetainLatestEpochOnClearAllEarliestAndUpdateItsOffset ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When reset to offset beyond last epoch
cache . clearAndFlushEarliest ( o ffset = 15 )
cache . truncateFromStart ( startO ffset = 15 )
// Then update the last
assertEquals ( ListBuffer ( EpochEntry ( 4 , 15 ) ) , cache . epochEntries )
@ -566,31 +475,25 @@ class LeaderEpochFileCacheTest {
@@ -566,31 +475,25 @@ class LeaderEpochFileCacheTest {
@Test
def shouldDropEntriesBetweenEpochBoundaryWhenRemovingNewest ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When reset to offset BETWEEN epoch boundaries
cache . clearAndFlushLatest ( o ffset = 9 )
cache . truncateFromEnd ( endO ffset = 9 )
// Then should keep the preceding epochs
assertEquals ( 3 , cache . latestEpoch ( ) )
assertEquals ( 3 , cache . latestEpoch )
assertEquals ( ListBuffer ( EpochEntry ( 2 , 6 ) , EpochEntry ( 3 , 8 ) ) , cache . epochEntries )
}
@Test
def shouldClearAllEntries ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When
cache . clearAndFlush ( )
@ -601,16 +504,13 @@ class LeaderEpochFileCacheTest {
@@ -601,16 +504,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldNotResetEpochHistoryHeadIfUndefinedPassed ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When reset to offset on epoch boundary
cache . clearAndFlushLatest ( o ffset = UNDEFINED_EPOCH_OFFSET )
cache . truncateFromEnd ( endO ffset = UNDEFINED_EPOCH_OFFSET )
// Then should do nothing
assertEquals ( 3 , cache . epochEntries . size )
@ -618,16 +518,13 @@ class LeaderEpochFileCacheTest {
@@ -618,16 +518,13 @@ class LeaderEpochFileCacheTest {
@Test
def shouldNotResetEpochHistoryTailIfUndefinedPassed ( ) : Unit = {
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// Given
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
cache . assign ( epoch = 2 , offset = 6 )
cache . assign ( epoch = 3 , offset = 8 )
cache . assign ( epoch = 4 , offset = 11 )
cache . assign ( epoch = 2 , startOffset = 6 )
cache . assign ( epoch = 3 , startOffset = 8 )
cache . assign ( epoch = 4 , startOffset = 11 )
// When reset to offset on epoch boundary
cache . clearAndFlushEarliest ( o ffset = UNDEFINED_EPOCH_OFFSET )
cache . truncateFromEnd ( endO ffset = UNDEFINED_EPOCH_OFFSET )
// Then should do nothing
assertEquals ( 3 , cache . epochEntries . size )
@ -635,54 +532,26 @@ class LeaderEpochFileCacheTest {
@@ -635,54 +532,26 @@ class LeaderEpochFileCacheTest {
@Test
def shouldFetchLatestEpochOfEmptyCache ( ) : Unit = {
// Given
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// When
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// Then
assertEquals ( - 1 , cache . latestEpoch )
}
@Test
def shouldFetchEndOffsetOfEmptyCache ( ) : Unit = {
// Given
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// When
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// Then
assertEquals ( ( UNDEFINED_EPOCH , UNDEFINED_EPOCH_OFFSET ) , cache . endOffsetFor ( 7 ) )
}
@Test
def shouldClearEarliestOnEmptyCache ( ) : Unit = {
// Given
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// When
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// Then
cache . clearAndFlushEarlies t( 7 )
cache . truncateFromStart ( 7 )
}
@Test
def shouldClearLatestOnEmptyCache ( ) : Unit = {
// Given
def leoFinder ( ) = new LogOffsetMetadata ( 0 )
// When
val cache = new LeaderEpochFileCache ( tp , ( ) => leoFinder , checkpoint )
// Then
cache . clearAndFlushLatest ( 7 )
cache . truncateFromEnd ( 7 )
}
@Before
def setUp ( ) {
checkpoint = new LeaderEpochCheckpointFile ( TestUtils . tempFile ( ) )
}
}