@ -18,22 +18,29 @@ package kafka.server
@@ -18,22 +18,29 @@ package kafka.server
import java.util.Optional
import kafka.api.Request
import kafka.cluster. { BrokerEndPoint , Partition , Replica }
import kafka.log.LogManager
import kafka.server.AbstractFetcherThread.ResultWithPartitions
import kafka.server.QuotaFactory.UnboundedQuota
import kafka.utils. { DelayedItem , TestUtils }
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.KafkaStorageException
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests. { EpochEndOffset , OffsetsForLeaderEpochRequest }
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.EpochEndOffset. { UNDEFINED_EPOCH , UNDEFINED_EPOCH_OFFSET }
import org.apache.kafka.common.requests. { EpochEndOffset , FetchRequest , IsolationLevel , OffsetsForLeaderEpochRequest }
import org.easymock.EasyMock._
import org.easymock. { Capture , CaptureType , EasyMock , IAnswer }
import org.junit.Assert._
import org.junit.Test
import org.mockito.Mockito. { doNothing , when }
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
import org.mockito. { ArgumentCaptor , ArgumentMatchers , Mockito }
import scala.collection.JavaConverters._
import scala.collection. { Map , Seq }
import scala.collection.JavaConverters._
class ReplicaAlterLogDirsThreadTest {
@ -44,6 +51,199 @@ class ReplicaAlterLogDirsThreadTest {
@@ -44,6 +51,199 @@ class ReplicaAlterLogDirsThreadTest {
OffsetAndEpoch ( offset = fetchOffset , leaderEpoch = leaderEpoch )
}
@Test
def shouldNotAddPartitionIfFutureLogIsNotDefined ( ) : Unit = {
val brokerId = 1
val config = KafkaConfig . fromProps ( TestUtils . createBrokerConfig ( brokerId , "localhost:1234" ) )
val replicaManager = Mockito . mock ( classOf [ ReplicaManager ] )
val quotaManager = Mockito . mock ( classOf [ ReplicationQuotaManager ] )
when ( replicaManager . futureLogExists ( t1p0 ) ) . thenReturn ( false )
val endPoint = new BrokerEndPoint ( 0 , "localhost" , 1000 )
val thread = new ReplicaAlterLogDirsThread (
"alter-logs-dirs-thread" ,
sourceBroker = endPoint ,
brokerConfig = config ,
replicaMgr = replicaManager ,
quota = quotaManager ,
brokerTopicStats = new BrokerTopicStats )
val addedPartitions = thread . addPartitions ( Map ( t1p0 -> offsetAndEpoch ( 0L ) ) )
assertEquals ( Set . empty , addedPartitions )
assertEquals ( 0 , thread . partitionCount ( ) )
assertEquals ( None , thread . fetchState ( t1p0 ) )
}
@Test
def shouldUpdateLeaderEpochAfterFencedEpochError ( ) : Unit = {
val brokerId = 1
val config = KafkaConfig . fromProps ( TestUtils . createBrokerConfig ( brokerId , "localhost:1234" ) )
val partition = Mockito . mock ( classOf [ Partition ] )
val replicaManager = Mockito . mock ( classOf [ ReplicaManager ] )
val quotaManager = Mockito . mock ( classOf [ ReplicationQuotaManager ] )
val futureReplica = Mockito . mock ( classOf [ Replica ] )
val leaderEpoch = 5
val logEndOffset = 0
when ( replicaManager . futureLocalReplicaOrException ( t1p0 ) ) . thenReturn ( futureReplica )
when ( replicaManager . futureLogExists ( t1p0 ) ) . thenReturn ( true )
when ( replicaManager . nonOfflinePartition ( t1p0 ) ) . thenReturn ( Some ( partition ) )
when ( replicaManager . getPartitionOrException ( t1p0 , expectLeader = false ) ) . thenReturn ( partition )
when ( replicaManager . getPartition ( t1p0 ) ) . thenReturn ( Some ( partition ) )
when ( quotaManager . isQuotaExceeded ) . thenReturn ( false )
when ( partition . lastOffsetForLeaderEpoch ( Optional . empty ( ) , leaderEpoch , fetchOnlyFromLeader = false ) )
. thenReturn ( new EpochEndOffset ( leaderEpoch , logEndOffset ) )
when ( partition . futureLocalReplicaOrException ) . thenReturn ( futureReplica )
doNothing ( ) . when ( partition ) . truncateTo ( offset = 0 , isFuture = true )
when ( partition . maybeReplaceCurrentWithFutureReplica ( ) ) . thenReturn ( true )
when ( futureReplica . logStartOffset ) . thenReturn ( 0L )
when ( futureReplica . logEndOffset ) . thenReturn ( LogOffsetMetadata ( 0L ) )
when ( futureReplica . latestEpoch ) . thenReturn ( None )
val fencedRequestData = new FetchRequest . PartitionData ( 0L , 0L ,
config . replicaFetchMaxBytes , Optional . of ( leaderEpoch - 1 ) )
val fencedResponseData = FetchPartitionData (
error = Errors . FENCED_LEADER_EPOCH ,
highWatermark = - 1 ,
logStartOffset = - 1 ,
records = MemoryRecords . EMPTY ,
lastStableOffset = None ,
abortedTransactions = None )
mockFetchFromCurrentLog ( t1p0 , fencedRequestData , config , replicaManager , fencedResponseData )
val endPoint = new BrokerEndPoint ( 0 , "localhost" , 1000 )
val thread = new ReplicaAlterLogDirsThread (
"alter-logs-dirs-thread" ,
sourceBroker = endPoint ,
brokerConfig = config ,
replicaMgr = replicaManager ,
quota = quotaManager ,
brokerTopicStats = new BrokerTopicStats )
// Initially we add the partition with an older epoch which results in an error
thread . addPartitions ( Map ( t1p0 -> offsetAndEpoch ( fetchOffset = 0L , leaderEpoch - 1 ) ) )
assertTrue ( thread . fetchState ( t1p0 ) . isDefined )
assertEquals ( 1 , thread . partitionCount ( ) )
thread . doWork ( )
assertEquals ( None , thread . fetchState ( t1p0 ) )
assertEquals ( 0 , thread . partitionCount ( ) )
// Next we update the epoch and assert that we can continue
thread . addPartitions ( Map ( t1p0 -> offsetAndEpoch ( fetchOffset = 0L , leaderEpoch ) ) )
assertEquals ( Some ( leaderEpoch ) , thread . fetchState ( t1p0 ) . map ( _ . currentLeaderEpoch ) )
assertEquals ( 1 , thread . partitionCount ( ) )
val requestData = new FetchRequest . PartitionData ( 0L , 0L ,
config . replicaFetchMaxBytes , Optional . of ( leaderEpoch ) )
val responseData = FetchPartitionData (
error = Errors . NONE ,
highWatermark = 0L ,
logStartOffset = 0L ,
records = MemoryRecords . EMPTY ,
lastStableOffset = None ,
abortedTransactions = None )
mockFetchFromCurrentLog ( t1p0 , requestData , config , replicaManager , responseData )
thread . doWork ( )
assertEquals ( None , thread . fetchState ( t1p0 ) )
assertEquals ( 0 , thread . partitionCount ( ) )
}
@Test
def shouldReplaceCurrentLogDirWhenCaughtUp ( ) : Unit = {
val brokerId = 1
val config = KafkaConfig . fromProps ( TestUtils . createBrokerConfig ( brokerId , "localhost:1234" ) )
val partition = Mockito . mock ( classOf [ Partition ] )
val replicaManager = Mockito . mock ( classOf [ ReplicaManager ] )
val quotaManager = Mockito . mock ( classOf [ ReplicationQuotaManager ] )
val futureReplica = Mockito . mock ( classOf [ Replica ] )
val leaderEpoch = 5
val logEndOffset = 0
when ( replicaManager . futureLocalReplicaOrException ( t1p0 ) ) . thenReturn ( futureReplica )
when ( replicaManager . futureLogExists ( t1p0 ) ) . thenReturn ( true )
when ( replicaManager . nonOfflinePartition ( t1p0 ) ) . thenReturn ( Some ( partition ) )
when ( replicaManager . getPartitionOrException ( t1p0 , expectLeader = false ) ) . thenReturn ( partition )
when ( replicaManager . getPartition ( t1p0 ) ) . thenReturn ( Some ( partition ) )
when ( quotaManager . isQuotaExceeded ) . thenReturn ( false )
when ( partition . lastOffsetForLeaderEpoch ( Optional . empty ( ) , leaderEpoch , fetchOnlyFromLeader = false ) )
. thenReturn ( new EpochEndOffset ( leaderEpoch , logEndOffset ) )
when ( partition . futureLocalReplicaOrException ) . thenReturn ( futureReplica )
doNothing ( ) . when ( partition ) . truncateTo ( offset = 0 , isFuture = true )
when ( partition . maybeReplaceCurrentWithFutureReplica ( ) ) . thenReturn ( true )
when ( futureReplica . logStartOffset ) . thenReturn ( 0L )
when ( futureReplica . logEndOffset ) . thenReturn ( LogOffsetMetadata ( 0L ) )
when ( futureReplica . latestEpoch ) . thenReturn ( None )
val requestData = new FetchRequest . PartitionData ( 0L , 0L ,
config . replicaFetchMaxBytes , Optional . of ( leaderEpoch ) )
val responseData = FetchPartitionData (
error = Errors . NONE ,
highWatermark = 0L ,
logStartOffset = 0L ,
records = MemoryRecords . EMPTY ,
lastStableOffset = None ,
abortedTransactions = None )
mockFetchFromCurrentLog ( t1p0 , requestData , config , replicaManager , responseData )
val endPoint = new BrokerEndPoint ( 0 , "localhost" , 1000 )
val thread = new ReplicaAlterLogDirsThread (
"alter-logs-dirs-thread" ,
sourceBroker = endPoint ,
brokerConfig = config ,
replicaMgr = replicaManager ,
quota = quotaManager ,
brokerTopicStats = new BrokerTopicStats )
thread . addPartitions ( Map ( t1p0 -> offsetAndEpoch ( fetchOffset = 0L , leaderEpoch ) ) )
assertTrue ( thread . fetchState ( t1p0 ) . isDefined )
assertEquals ( 1 , thread . partitionCount ( ) )
thread . doWork ( )
assertEquals ( None , thread . fetchState ( t1p0 ) )
assertEquals ( 0 , thread . partitionCount ( ) )
}
private def mockFetchFromCurrentLog ( topicPartition : TopicPartition ,
requestData : FetchRequest . PartitionData ,
config : KafkaConfig ,
replicaManager : ReplicaManager ,
responseData : FetchPartitionData ) : Unit = {
val callbackCaptor : ArgumentCaptor [ Seq [ ( TopicPartition , FetchPartitionData ) ] => Unit ] =
ArgumentCaptor . forClass ( classOf [ Seq [ ( TopicPartition , FetchPartitionData ) ] => Unit ] )
when ( replicaManager . fetchMessages (
timeout = ArgumentMatchers . eq ( 0L ) ,
replicaId = ArgumentMatchers . eq ( Request . FutureLocalReplicaId ) ,
fetchMinBytes = ArgumentMatchers . eq ( 0 ) ,
fetchMaxBytes = ArgumentMatchers . eq ( config . replicaFetchResponseMaxBytes ) ,
hardMaxBytesLimit = ArgumentMatchers . eq ( false ) ,
fetchInfos = ArgumentMatchers . eq ( Seq ( topicPartition -> requestData ) ) ,
quota = ArgumentMatchers . eq ( UnboundedQuota ) ,
responseCallback = callbackCaptor . capture ( ) ,
isolationLevel = ArgumentMatchers . eq ( IsolationLevel . READ_UNCOMMITTED )
) ) . thenAnswer ( new Answer [ Any ] {
override def answer ( invocation : InvocationOnMock ) : Unit = {
callbackCaptor . getValue . apply ( Seq ( ( topicPartition , responseData ) ) )
}
} )
}
@Test
def issuesEpochRequestFromLocalReplica ( ) : Unit = {
val config = KafkaConfig . fromProps ( TestUtils . createBrokerConfig ( 1 , "localhost:1234" ) )
@ -171,7 +371,9 @@ class ReplicaAlterLogDirsThreadTest {
@@ -171,7 +371,9 @@ class ReplicaAlterLogDirsThreadTest {
expect ( replicaManager . getPartitionOrException ( t1p1 , expectLeader = false ) )
. andStubReturn ( partitionT1p1 )
expect ( replicaManager . futureLocalReplicaOrException ( t1p0 ) ) . andStubReturn ( futureReplicaT1p0 )
expect ( replicaManager . futureLogExists ( t1p0 ) ) . andStubReturn ( true )
expect ( replicaManager . futureLocalReplicaOrException ( t1p1 ) ) . andStubReturn ( futureReplicaT1p1 )
expect ( replicaManager . futureLogExists ( t1p1 ) ) . andStubReturn ( true )
expect ( partitionT1p0 . truncateTo ( capture ( truncateCaptureT1p0 ) , anyBoolean ( ) ) ) . anyTimes ( )
expect ( partitionT1p1 . truncateTo ( capture ( truncateCaptureT1p1 ) , anyBoolean ( ) ) ) . anyTimes ( )
@ -244,6 +446,7 @@ class ReplicaAlterLogDirsThreadTest {
@@ -244,6 +446,7 @@ class ReplicaAlterLogDirsThreadTest {
expect ( replicaManager . getPartitionOrException ( t1p0 , expectLeader = false ) )
. andStubReturn ( partition )
expect ( replicaManager . futureLocalReplicaOrException ( t1p0 ) ) . andStubReturn ( futureReplica )
expect ( replicaManager . futureLogExists ( t1p0 ) ) . andStubReturn ( true )
expect ( partition . truncateTo ( capture ( truncateToCapture ) , EasyMock . eq ( true ) ) ) . anyTimes ( )
expect ( futureReplica . logEndOffset ) . andReturn ( new LogOffsetMetadata ( futureReplicaLEO ) ) . anyTimes ( )
@ -314,6 +517,7 @@ class ReplicaAlterLogDirsThreadTest {
@@ -314,6 +517,7 @@ class ReplicaAlterLogDirsThreadTest {
. andStubReturn ( partition )
expect ( partition . truncateTo ( capture ( truncated ) , isFuture = EasyMock . eq ( true ) ) ) . anyTimes ( )
expect ( replicaManager . futureLocalReplicaOrException ( t1p0 ) ) . andStubReturn ( futureReplica )
expect ( replicaManager . futureLogExists ( t1p0 ) ) . andStubReturn ( true )
expect ( futureReplica . logEndOffset ) . andReturn ( new LogOffsetMetadata ( futureReplicaLEO ) ) . anyTimes ( )
expect ( replicaManager . logManager ) . andReturn ( logManager ) . anyTimes ( )
@ -369,6 +573,7 @@ class ReplicaAlterLogDirsThreadTest {
@@ -369,6 +573,7 @@ class ReplicaAlterLogDirsThreadTest {
expect ( partition . truncateTo ( capture ( truncated ) , isFuture = EasyMock . eq ( true ) ) ) . once ( )
expect ( replicaManager . futureLocalReplicaOrException ( t1p0 ) ) . andStubReturn ( futureReplica )
expect ( replicaManager . futureLogExists ( t1p0 ) ) . andStubReturn ( true )
expect ( futureReplica . latestEpoch ) . andStubReturn ( Some ( futureReplicaLeaderEpoch ) )
expect ( futureReplica . endOffsetForEpoch ( futureReplicaLeaderEpoch ) ) . andReturn (
Some ( OffsetAndEpoch ( futureReplicaLEO , futureReplicaLeaderEpoch ) ) )
@ -452,6 +657,7 @@ class ReplicaAlterLogDirsThreadTest {
@@ -452,6 +657,7 @@ class ReplicaAlterLogDirsThreadTest {
expect ( partition . truncateTo ( futureReplicaLEO , isFuture = true ) ) . once ( )
expect ( replicaManager . futureLocalReplicaOrException ( t1p0 ) ) . andStubReturn ( futureReplica )
expect ( replicaManager . futureLogExists ( t1p0 ) ) . andStubReturn ( true )
expect ( futureReplica . latestEpoch ) . andStubReturn ( Some ( leaderEpoch ) )
expect ( futureReplica . logEndOffset ) . andReturn ( new LogOffsetMetadata ( futureReplicaLEO ) ) . anyTimes ( )
expect ( futureReplica . endOffsetForEpoch ( leaderEpoch ) ) . andReturn (
@ -604,11 +810,13 @@ class ReplicaAlterLogDirsThreadTest {
@@ -604,11 +810,13 @@ class ReplicaAlterLogDirsThreadTest {
expect ( replicaManager . futureLocalReplica ( t1p0 ) ) . andReturn ( Some ( futureReplica ) ) . anyTimes ( )
expect ( replicaManager . localReplicaOrException ( t1p0 ) ) . andReturn ( replicaT1p0 ) . anyTimes ( )
expect ( replicaManager . futureLocalReplicaOrException ( t1p0 ) ) . andReturn ( futureReplica ) . anyTimes ( )
expect ( replicaManager . futureLogExists ( t1p0 ) ) . andStubReturn ( true )
expect ( replicaManager . getPartition ( t1p0 ) ) . andReturn ( Some ( partition ) ) . anyTimes ( )
expect ( replicaManager . localReplica ( t1p1 ) ) . andReturn ( Some ( replicaT1p1 ) ) . anyTimes ( )
expect ( replicaManager . futureLocalReplica ( t1p1 ) ) . andReturn ( Some ( futureReplica ) ) . anyTimes ( )
expect ( replicaManager . localReplicaOrException ( t1p1 ) ) . andReturn ( replicaT1p1 ) . anyTimes ( )
expect ( replicaManager . futureLocalReplicaOrException ( t1p1 ) ) . andReturn ( futureReplica ) . anyTimes ( )
expect ( replicaManager . futureLogExists ( t1p1 ) ) . andStubReturn ( true )
expect ( replicaManager . getPartition ( t1p1 ) ) . andReturn ( Some ( partition ) ) . anyTimes ( )
}