@ -43,7 +43,7 @@ import org.apache.kafka.clients.admin.ConfigEntry.{ConfigSource, ConfigSynonym}
@@ -43,7 +43,7 @@ import org.apache.kafka.clients.admin.ConfigEntry.{ConfigSource, ConfigSynonym}
import org.apache.kafka.clients.admin._
import org.apache.kafka.clients.consumer. { ConsumerConfig , ConsumerRecord , ConsumerRecords , KafkaConsumer }
import org.apache.kafka.clients.producer. { KafkaProducer , ProducerRecord }
import org.apache.kafka.common. { ClusterResource , ClusterResourceListener , Reconfigurable , TopicPartition }
import org.apache.kafka.common. { ClusterResource , ClusterResourceListener , Reconfigurable , TopicPartition , TopicPartitionInfo }
import org.apache.kafka.common.config. { ConfigException , ConfigResource }
import org.apache.kafka.common.config.SslConfigs._
import org.apache.kafka.common.config.types.Password
@ -429,6 +429,62 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
@@ -429,6 +429,62 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
stopAndVerifyProduceConsume ( producerThread , consumerThread )
}
@Test
def testUncleanLeaderElectionEnable ( ) : Unit = {
val topic = "testtopic2"
TestUtils . createTopic ( zkClient , topic , 1 , replicationFactor = 2 , servers )
val producer = ProducerBuilder ( ) . maxRetries ( 1000 ) . acks ( 1 ) . build ( )
val consumer = ConsumerBuilder ( "unclean-leader-test" ) . enableAutoCommit ( false ) . topic ( topic ) . build ( )
verifyProduceConsume ( producer , consumer , numRecords = 10 , topic )
consumer . commitSync ( )
def partitionInfo : TopicPartitionInfo =
adminClients . head . describeTopics ( Collections . singleton ( topic ) ) . values . get ( topic ) . get ( ) . partitions ( ) . get ( 0 )
val partitionInfo0 = partitionInfo
assertEquals ( partitionInfo0 . replicas . get ( 0 ) , partitionInfo0 . leader )
val leaderBroker = servers . find ( _ . config . brokerId == partitionInfo0 . replicas . get ( 0 ) . id ) . get
val followerBroker = servers . find ( _ . config . brokerId == partitionInfo0 . replicas . get ( 1 ) . id ) . get
// Stop follower
followerBroker . shutdown ( )
followerBroker . awaitShutdown ( )
// Produce and consume some messages when the only follower is down , this should succeed since MinIsr is 1
verifyProduceConsume ( producer , consumer , numRecords = 10 , topic )
consumer . commitSync ( )
// Shutdown leader and startup follower
leaderBroker . shutdown ( )
leaderBroker . awaitShutdown ( )
followerBroker . startup ( )
val controller = servers . find ( _ . config . brokerId == TestUtils . waitUntilControllerElected ( zkClient ) ) . get
// Verify that new leader is not elected with unclean leader disabled since there are no ISRs
TestUtils . waitUntilTrue ( ( ) => partitionInfo . leader == null , "Unclean leader elected" )
// Enable unclean leader election
val newProps = new Properties
newProps . put ( KafkaConfig . UncleanLeaderElectionEnableProp , "true" )
TestUtils . alterConfigs ( servers , adminClients . head , newProps , perBrokerConfig = false ) . all . get
waitForConfigOnServer ( controller , KafkaConfig . UncleanLeaderElectionEnableProp , "true" )
// Verify that the old follower with missing records is elected as the new leader
val ( newLeader , elected ) = TestUtils . computeUntilTrue ( partitionInfo . leader ) ( leader => leader != null )
assertTrue ( "Unclean leader not elected" , elected )
assertEquals ( followerBroker . config . brokerId , newLeader . id )
// New leader doesn 't have the last 10 records committed on the old leader that have already been consumed .
// With unclean leader election enabled , we should be able to produce to the new leader . The first 10 records
// produced will not be consumed since they have offsets less than the consumer 's committed offset .
// Next 10 records produced should be consumed .
( 1 to 10 ) . map ( i => new ProducerRecord ( topic , s" key $i " , s" value $i " ) )
. map ( producer . send )
. map ( _ . get ( 10 , TimeUnit . SECONDS ) )
verifyProduceConsume ( producer , consumer , numRecords = 10 , topic )
consumer . commitSync ( )
}
@Test
def testThreadPoolResize ( ) : Unit = {
val requestHandlerPrefix = "kafka-request-handler-"
@ -1272,12 +1328,14 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
@@ -1272,12 +1328,14 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
private case class ProducerBuilder ( ) extends ClientBuilder [ KafkaProducer [ String , String ] ] {
private var _retries = 0
private var _acks = - 1
def maxRetries ( retries : Int ) : ProducerBuilder = { _retries = retries ; this }
def acks ( acks : Int ) : ProducerBuilder = { _acks = acks ; this }
override def build ( ) : KafkaProducer [ String , String ] = {
val producer = TestUtils . createNewProducer ( bootstrapServers ,
acks = - 1 ,
acks = _acks ,
retries = _retries ,
securityProtocol = _securityProtocol ,
trustStoreFile = Some ( trustStoreFile1 ) ,