@ -35,6 +35,7 @@ import org.apache.kafka.common.errors.TimeoutException
@@ -35,6 +35,7 @@ import org.apache.kafka.common.errors.TimeoutException
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.serialization.StringDeserializer
import org.apache.kafka.clients.admin. { AdminClient , AdminClientConfig }
import org.junit.Assert._
class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
@ -238,7 +239,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
@@ -238,7 +239,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
// message production and consumption should both fail while leader is down
try {
produceMessage ( servers , topic , "third" )
produceMessage ( servers , topic , "third" , deliveryTimeoutMs = 1000 , requestTimeoutMs = 1000 )
fail ( "Message produced while leader is down should fail, but it succeeded" )
} catch {
case e : ExecutionException if e.getCause.isInstanceOf [ TimeoutException ] => // expected
@ -280,4 +281,73 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
@@ -280,4 +281,73 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
TestUtils . consumeRecords ( consumer , numMessages ) . map ( _ . value )
} finally consumer . close ( )
}
@Test
def testTopicUncleanLeaderElectionEnable ( ) : Unit = {
// unclean leader election is disabled by default
startBrokers ( Seq ( configProps1 , configProps2 ) )
// create topic with 1 partition , 2 replicas , one on each broker
adminZkClient . createOrUpdateTopicPartitionAssignmentPathInZK ( topic , Map ( partitionId -> Seq ( brokerId1 , brokerId2 ) ) )
// wait until leader is elected
val leaderId = waitUntilLeaderIsElectedOrChanged ( zkClient , topic , partitionId )
// the non - leader broker is the follower
val followerId = if ( leaderId == brokerId1 ) brokerId2 else brokerId1
produceMessage ( servers , topic , "first" )
waitUntilMetadataIsPropagated ( servers , topic , partitionId )
assertEquals ( List ( "first" ) , consumeAllMessages ( topic , 1 ) )
// shutdown follower server
servers . filter ( server => server . config . brokerId == followerId ) . map ( server => shutdownServer ( server ) )
produceMessage ( servers , topic , "second" )
assertEquals ( List ( "first" , "second" ) , consumeAllMessages ( topic , 2 ) )
// remove any previous unclean election metric
servers . map ( server => server . kafkaController . controllerContext . stats . removeMetric ( "UncleanLeaderElectionsPerSec" ) )
// shutdown leader and then restart follower
servers . filter ( server => server . config . brokerId == leaderId ) . map ( server => shutdownServer ( server ) )
val followerServer = servers . find ( _ . config . brokerId == followerId ) . get
followerServer . startup ( )
assertEquals ( 0 , followerServer . kafkaController . controllerContext . stats . uncleanLeaderElectionRate . count ( ) )
// message production and consumption should both fail while leader is down
try {
produceMessage ( servers , topic , "third" , deliveryTimeoutMs = 1000 , requestTimeoutMs = 1000 )
fail ( "Message produced while leader is down should fail, but it succeeded" )
} catch {
case e : ExecutionException if e.getCause.isInstanceOf [ TimeoutException ] => // expected
}
assertEquals ( List . empty [ String ] , consumeAllMessages ( topic , 0 ) )
// Enable unclean leader election for topic
val adminClient = createAdminClient ( )
val newProps = new Properties
newProps . put ( KafkaConfig . UncleanLeaderElectionEnableProp , "true" )
TestUtils . alterTopicConfigs ( adminClient , topic , newProps ) . all . get
adminClient . close ( )
// wait until new leader is ( uncleanly ) elected
waitUntilLeaderIsElectedOrChanged ( zkClient , topic , partitionId , newLeaderOpt = Some ( followerId ) )
assertEquals ( 1 , followerServer . kafkaController . controllerContext . stats . uncleanLeaderElectionRate . count ( ) )
produceMessage ( servers , topic , "third" )
// second message was lost due to unclean election
assertEquals ( List ( "first" , "third" ) , consumeAllMessages ( topic , 2 ) )
}
private def createAdminClient ( ) : AdminClient = {
val config = new Properties
val bootstrapServers = TestUtils . bootstrapServers ( servers , new ListenerName ( "PLAINTEXT" ) )
config . put ( AdminClientConfig . BOOTSTRAP_SERVERS_CONFIG , bootstrapServers )
config . put ( AdminClientConfig . METADATA_MAX_AGE_CONFIG , "10" )
AdminClient . create ( config )
}
}