@ -21,23 +21,24 @@ import java.io.Closeable
@@ -21,23 +21,24 @@ import java.io.Closeable
import java.util. { Collections , HashMap , List }
import kafka.admin.ReassignPartitionsCommand._
import kafka.server. { KafkaConfig , KafkaServer }
import kafka.api.KAFKA_2_7_IV1
import kafka.server. { IsrChangePropagationConfig , KafkaConfig , KafkaServer , ReplicaManager }
import kafka.utils.Implicits._
import kafka.utils.TestUtils
import kafka.zk. { KafkaZkClient , ZooKeeperTestHarness }
import org.apache.kafka.clients.admin. { Admin , AdminClientConfig , AlterConfigOp , ConfigEntry , DescribeLogDirsResult , NewTopic }
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common. { TopicPartition , TopicPartitionReplica }
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Utils
import org.junit.rules.Timeout
import org.apache.kafka.common. { TopicPartition , TopicPartitionReplica }
import org.junit.Assert. { assertEquals , assertFalse , assertTrue }
import org.junit.rules.Timeout
import org.junit. { After , Rule , Test }
import scala.collection.Map
import scala.collection. { Map , Seq , mutable }
import scala.jdk.CollectionConverters._
import scala.collection. { Seq , mutable }
class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
@Rule
@ -45,10 +46,6 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
@@ -45,10 +46,6 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
var cluster : ReassignPartitionsTestCluster = null
def generateConfigs : Seq [ KafkaConfig ] = {
TestUtils . createBrokerConfigs ( 5 , zkConnect ) . map ( KafkaConfig . fromProps )
}
@After
override def tearDown ( ) : Unit = {
Utils . closeQuietly ( cluster , "ReassignPartitionsTestCluster" )
@ -60,13 +57,53 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
@@ -60,13 +57,53 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
brokerId -> brokerLevelThrottles . map ( throttle => ( throttle , - 1L ) ) . toMap
} . toMap
/* *
* Test running a quick reassignment .
*/
@Test
def testReassignment ( ) : Unit = {
cluster = new ReassignPartitionsTestCluster ( zkConnect )
cluster . setup ( )
executeAndVerifyReassignment ( )
}
@Test
def testReassignmentWithAlterIsrDisabled ( ) : Unit = {
// Test reassignment when the IBP is on an older version which does not use
// the `AlterIsr` API . In this case , the controller will register individual
// watches for each reassigning partition so that the reassignment can be
// completed as soon as the ISR is expanded .
val configOverrides = Map ( KafkaConfig . InterBrokerProtocolVersionProp -> KAFKA_2_7_IV1 . version )
cluster = new ReassignPartitionsTestCluster ( zkConnect , configOverrides = configOverrides )
cluster . setup ( )
executeAndVerifyReassignment ( )
}
@Test
def testReassignmentCompletionDuringPartialUpgrade ( ) : Unit = {
// Test reassignment during a partial upgrade when some brokers are relying on
// `AlterIsr` and some rely on the old notification logic through Zookeeper .
// In this test case , broker 0 starts up first on the latest IBP and is typically
// elected as controller . The three remaining brokers start up on the older IBP .
// We want to ensure that reassignment can still complete through the ISR change
// notification path even though the controller expects `AlterIsr` .
// Override change notification settings so that test is not delayed by ISR
// change notification delay
ReplicaManager . DefaultIsrPropagationConfig = IsrChangePropagationConfig (
checkIntervalMs = 500 ,
lingerMs = 100 ,
maxDelayMs = 500
)
val oldIbpConfig = Map ( KafkaConfig . InterBrokerProtocolVersionProp -> KAFKA_2_7_IV1 . version )
val brokerConfigOverrides = Map ( 1 -> oldIbpConfig , 2 -> oldIbpConfig , 3 -> oldIbpConfig )
cluster = new ReassignPartitionsTestCluster ( zkConnect , brokerConfigOverrides = brokerConfigOverrides )
cluster . setup ( )
executeAndVerifyReassignment ( )
}
def executeAndVerifyReassignment ( ) : Unit = {
val assignment = """{"version":1,"partitions":""" +
"""[{"topic":"foo","partition":0,"replicas":[0,1,3],"log_dirs":["any","any","any"]},""" +
"""{"topic":"bar","partition":0,"replicas":[3,2,0],"log_dirs":["any","any","any"]}""" +
@ -594,7 +631,11 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
@@ -594,7 +631,11 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
}
}
class ReassignPartitionsTestCluster ( val zkConnect : String ) extends Closeable {
class ReassignPartitionsTestCluster (
val zkConnect : String ,
configOverrides : Map [ String , String ] = Map . empty ,
brokerConfigOverrides : Map [ Int , Map [ String , String ] ] = Map . empty
) extends Closeable {
val brokers = Map (
0 -> "rack0" ,
1 -> "rack0" ,
@ -622,6 +663,12 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
@@ -622,6 +663,12 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
// Don 't move partition leaders automatically .
config . setProperty ( KafkaConfig . AutoLeaderRebalanceEnableProp , "false" )
config . setProperty ( KafkaConfig . ReplicaLagTimeMaxMsProp , "1000" )
configOverrides . forKeyValue ( config . setProperty )
brokerConfigOverrides . get ( brokerId ) . foreach { overrides =>
overrides . forKeyValue ( config . setProperty )
}
config
} . toBuffer
@ -637,9 +684,8 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
@@ -637,9 +684,8 @@ class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
}
def createServers ( ) : Unit = {
brokers . keySet . foreach {
case brokerId =>
servers += TestUtils . createServer ( KafkaConfig ( brokerConfigs ( brokerId ) ) )
brokers . keySet . foreach { brokerId =>
servers += TestUtils . createServer ( KafkaConfig ( brokerConfigs ( brokerId ) ) )
}
}