KAFKA-8345 (KIP-455): Controller and KafkaApi changes (part 3/4) (#7128)

Implement the revisions to the controller state machine and reassignment logic needed for KIP-455.

Add the addingReplicas and removingReplicas field to the topics ZNode.

Deprecate the methods initiating a reassignment via direct ZK access in KafkaZkClient.

Add ControllerContextTest, and add some test cases to ReassignPartitionsClusterTest.

Add a note to upgrade.html recommending not initiating reassignments during an upgrade.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Viktor Somogyi <viktorsomogyi@gmail.com>
This commit is contained in:
Stanislav Kozlovski 2019-09-10 22:19:44 -07:00 committed by Colin Patrick McCabe
parent 41b89a6ecf
commit 18d4e57f6e
27 changed files with 1722 additions and 327 deletions

View File

@ -371,8 +371,8 @@ object TopicCommand extends Logging {
}
println("WARNING: If partitions are increased for a topic that has a key, the partition " +
"logic or ordering of the messages will be affected")
val existingAssignment = zkClient.getReplicaAssignmentForTopics(immutable.Set(topic)).map {
case (topicPartition, replicas) => topicPartition.partition -> replicas
val existingAssignment = zkClient.getFullReplicaAssignmentForTopics(immutable.Set(topic)).map {
case (topicPartition, assignment) => topicPartition.partition -> assignment
}
if (existingAssignment.isEmpty)
throw new InvalidTopicException(s"The topic $topic does not exist")
@ -401,14 +401,15 @@ object TopicCommand extends Logging {
val configs = adminZkClient.fetchEntityConfig(ConfigType.Topic, topic).asScala
if (!opts.reportOverriddenConfigs || configs.nonEmpty) {
val numPartitions = topicPartitionAssignment.size
val replicationFactor = topicPartitionAssignment.head._2.size
val replicationFactor = topicPartitionAssignment.head._2.replicas.size
val config = new JConfig(configs.map{ case (k, v) => new ConfigEntry(k, v) }.asJavaCollection)
val topicDesc = TopicDescription(topic, numPartitions, replicationFactor, config, markedForDeletion)
topicDesc.printDescription()
}
}
if (describeOptions.describePartitions) {
for ((partitionId, assignedReplicas) <- topicPartitionAssignment.toSeq.sortBy(_._1)) {
for ((partitionId, replicaAssignment) <- topicPartitionAssignment.toSeq.sortBy(_._1)) {
val assignedReplicas = replicaAssignment.replicas
val tp = new TopicPartition(topic, partitionId)
val (leaderOpt, isr) = zkClient.getTopicPartitionState(tp).map(_.leaderAndIsr) match {
case Some(leaderAndIsr) => (leaderAndIsr.leaderOpt, leaderAndIsr.isr)

View File

@ -376,7 +376,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int],
topicPartition: TopicPartition,
leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch,
replicas: Seq[Int],
replicaAssignment: PartitionReplicaAssignment,
isNew: Boolean): Unit = {
brokerIds.filter(_ >= 0).foreach { brokerId =>
@ -387,7 +387,9 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch,
leaderIsrAndControllerEpoch.leaderAndIsr.isr.map(Integer.valueOf).asJava,
leaderIsrAndControllerEpoch.leaderAndIsr.zkVersion,
replicas.map(Integer.valueOf).asJava,
replicaAssignment.replicas.map(Integer.valueOf).asJava,
replicaAssignment.addingReplicas.map(Integer.valueOf).asJava,
replicaAssignment.removingReplicas.map(Integer.valueOf).asJava,
isNew || alreadyNew))
}

View File

@ -22,6 +22,45 @@ import org.apache.kafka.common.TopicPartition
import scala.collection.{Map, Seq, Set, mutable}
object PartitionReplicaAssignment {
def fromOldAndNewReplicas(oldReplicas: Seq[Int], newReplicas: Seq[Int]): PartitionReplicaAssignment = {
val fullReplicaSet = (newReplicas ++ oldReplicas).distinct
PartitionReplicaAssignment(
fullReplicaSet,
fullReplicaSet.filterNot(oldReplicas.contains(_)),
fullReplicaSet.filterNot(newReplicas.contains(_))
)
}
}
case class PartitionReplicaAssignment(replicas: Seq[Int], addingReplicas: Seq[Int], removingReplicas: Seq[Int]) {
def isBeingReassigned: Boolean = {
addingReplicas.nonEmpty || removingReplicas.nonEmpty
}
/**
* Returns the partition replica assignment previous to this one.
* It is different than this one only when the partition is undergoing reassignment
* Note that this will not preserve the original ordering
*/
def previousAssignment: PartitionReplicaAssignment = {
PartitionReplicaAssignment(
replicas.filterNot(addingReplicas.contains(_)),
Seq(),
Seq()
)
}
/**
* Returns the target replica assignment for this partition.
* This is different than the `replicas` variable only when there is a reassignment going on
*/
def targetReplicas: Seq[Int] = replicas.filterNot(removingReplicas.contains(_))
override def toString: String = s"PartitionReplicaAssignment(replicas: ${replicas.mkString(",")}, " +
s"addingReplicas: ${addingReplicas.mkString(",")}, removingReplicas: ${removingReplicas.mkString(",")})"
}
class ControllerContext {
val stats = new ControllerStats
var offlinePartitionCount = 0
@ -32,7 +71,7 @@ class ControllerContext {
var epochZkVersion: Int = KafkaController.InitialControllerEpochZkVersion
var allTopics: Set[String] = Set.empty
val partitionAssignments = mutable.Map.empty[String, mutable.Map[Int, Seq[Int]]]
val partitionAssignments = mutable.Map.empty[String, mutable.Map[Int, PartitionReplicaAssignment]]
val partitionLeadershipInfo = mutable.Map.empty[TopicPartition, LeaderIsrAndControllerEpoch]
val partitionsBeingReassigned = mutable.Map.empty[TopicPartition, ReassignedPartitionsContext]
val partitionStates = mutable.Map.empty[TopicPartition, PartitionState]
@ -63,12 +102,6 @@ class ControllerContext {
val topicsWithDeletionStarted = mutable.Set.empty[String]
val topicsIneligibleForDeletion = mutable.Set.empty[String]
def partitionReplicaAssignment(topicPartition: TopicPartition): Seq[Int] = {
partitionAssignments.getOrElse(topicPartition.topic, mutable.Map.empty)
.getOrElse(topicPartition.partition, Seq.empty)
}
private def clearTopicsState(): Unit = {
allTopics = Set.empty
partitionAssignments.clear()
@ -80,14 +113,55 @@ class ControllerContext {
replicaStates.clear()
}
def partitionReplicaAssignment(topicPartition: TopicPartition): Seq[Int] = {
partitionAssignments.getOrElse(topicPartition.topic, mutable.Map.empty)
.get(topicPartition.partition) match {
case Some(partitionAssignment) => partitionAssignment.replicas
case None => Seq.empty
}
}
def partitionFullReplicaAssignment(topicPartition: TopicPartition): PartitionReplicaAssignment = {
partitionAssignments.getOrElse(topicPartition.topic, mutable.Map.empty)
.get(topicPartition.partition) match {
case Some(partitionAssignment) => partitionAssignment
case None => PartitionReplicaAssignment(Seq(), Seq(), Seq())
}
}
def updatePartitionReplicaAssignment(topicPartition: TopicPartition, newReplicas: Seq[Int]): Unit = {
partitionAssignments.getOrElseUpdate(topicPartition.topic, mutable.Map.empty)
.put(topicPartition.partition, newReplicas)
val assignments = partitionAssignments.getOrElseUpdate(topicPartition.topic, mutable.Map.empty)
val newAssignment = assignments.get(topicPartition.partition) match {
case Some(partitionAssignment) =>
PartitionReplicaAssignment(
newReplicas,
partitionAssignment.addingReplicas,
partitionAssignment.removingReplicas
)
case None =>
PartitionReplicaAssignment(
newReplicas,
Seq.empty,
Seq.empty
)
}
updatePartitionFullReplicaAssignment(topicPartition, newAssignment)
}
def updatePartitionFullReplicaAssignment(topicPartition: TopicPartition, newAssignment: PartitionReplicaAssignment): Unit = {
val assignments = partitionAssignments.getOrElseUpdate(topicPartition.topic, mutable.Map.empty)
assignments.put(topicPartition.partition, newAssignment)
}
def partitionReplicaAssignmentForTopic(topic : String): Map[TopicPartition, Seq[Int]] = {
partitionAssignments.getOrElse(topic, Map.empty).map {
case (partition, replicas) => (new TopicPartition(topic, partition), replicas)
case (partition, assignment) => (new TopicPartition(topic, partition), assignment.replicas)
}.toMap
}
def partitionFullReplicaAssignmentForTopic(topic : String): Map[TopicPartition, PartitionReplicaAssignment] = {
partitionAssignments.getOrElse(topic, Map.empty).map {
case (partition, assignment) => (new TopicPartition(topic, partition), assignment)
}.toMap
}
@ -131,7 +205,7 @@ class ControllerContext {
def partitionsOnBroker(brokerId: Int): Set[TopicPartition] = {
partitionAssignments.flatMap {
case (topic, topicReplicaAssignment) => topicReplicaAssignment.filter {
case (_, replicas) => replicas.contains(brokerId)
case (_, partitionAssignment) => partitionAssignment.replicas.contains(brokerId)
}.map {
case (partition, _) => new TopicPartition(topic, partition)
}
@ -150,7 +224,7 @@ class ControllerContext {
brokerIds.flatMap { brokerId =>
partitionAssignments.flatMap {
case (topic, topicReplicaAssignment) => topicReplicaAssignment.collect {
case (partition, replicas) if replicas.contains(brokerId) =>
case (partition, partitionAssignment) if partitionAssignment.replicas.contains(brokerId) =>
PartitionAndReplica(new TopicPartition(topic, partition), brokerId)
}
}
@ -159,7 +233,7 @@ class ControllerContext {
def replicasForTopic(topic: String): Set[PartitionAndReplica] = {
partitionAssignments.getOrElse(topic, mutable.Map.empty).flatMap {
case (partition, replicas) => replicas.map(r => PartitionAndReplica(new TopicPartition(topic, partition), r))
case (partition, assignment) => assignment.replicas.map(r => PartitionAndReplica(new TopicPartition(topic, partition), r))
}.toSet
}
@ -183,10 +257,10 @@ class ControllerContext {
def onlineAndOfflineReplicas: (Set[PartitionAndReplica], Set[PartitionAndReplica]) = {
val onlineReplicas = mutable.Set.empty[PartitionAndReplica]
val offlineReplicas = mutable.Set.empty[PartitionAndReplica]
for ((topic, partitionReplicas) <- partitionAssignments;
(partitionId, replicas) <- partitionReplicas) {
for ((topic, partitionAssignments) <- partitionAssignments;
(partitionId, assignment) <- partitionAssignments) {
val partition = new TopicPartition(topic, partitionId)
for (replica <- replicas) {
for (replica <- assignment.replicas) {
val partitionAndReplica = PartitionAndReplica(partition, replica)
if (isReplicaOnline(replica, partition))
onlineReplicas.add(partitionAndReplica)

View File

@ -58,7 +58,7 @@ object ControllerState {
def value = 4
}
case object PartitionReassignment extends ControllerState {
case object AlterPartitionReassignment extends ControllerState {
def value = 5
}
@ -98,7 +98,11 @@ object ControllerState {
def value = 14
}
val values: Seq[ControllerState] = Seq(Idle, ControllerChange, BrokerChange, TopicChange, TopicDeletion,
PartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange, LeaderAndIsrResponseReceived,
LogDirChange, ControllerShutdown, UncleanLeaderElectionEnable, TopicUncleanLeaderElectionEnable)
case object ListPartitionReassignment extends ControllerState {
def value = 15
}
val values: Seq[ControllerState] = Seq(Idle, ControllerChange, BrokerChange, TopicChange, TopicDeletion,
AlterPartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange, LeaderAndIsrResponseReceived,
LogDirChange, ControllerShutdown, UncleanLeaderElectionEnable, TopicUncleanLeaderElectionEnable, ListPartitionReassignment)
}

View File

@ -22,7 +22,7 @@ import com.yammer.metrics.core.Gauge
import kafka.admin.AdminOperationException
import kafka.api._
import kafka.common._
import kafka.controller.KafkaController.ElectLeadersCallback
import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback}
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
import kafka.server._
import kafka.utils._
@ -54,6 +54,8 @@ object KafkaController extends Logging {
val InitialControllerEpochZkVersion = 0
type ElectLeadersCallback = Map[TopicPartition, Either[ApiError, Int]] => Unit
type ListReassignmentsCallback = Either[Map[TopicPartition, PartitionReplicaAssignment], ApiError] => Unit
type AlterReassignmentsCallback = Either[Map[TopicPartition, ApiError], ApiError] => Unit
}
class KafkaController(val config: KafkaConfig,
@ -534,104 +536,195 @@ class KafkaController(val config: KafkaConfig,
}
/**
* This callback is invoked by the reassigned partitions listener. When an admin command initiates a partition
* reassignment, it creates the /admin/reassign_partitions path that triggers the zookeeper listener.
* This callback is invoked:
* 1. By the AlterPartitionReassignments API
* 2. By the reassigned partitions listener which is triggered when the /admin/reassign/partitions znode is created
* 3. When an ongoing reassignment finishes - this is detected by a change in the partition's ISR znode
* 4. Whenever a new broker comes up which is part of an ongoing reassignment
* 5. On controller startup/failover
*
*
* Reassigning replicas for a partition goes through a few steps listed in the code.
* RAR = Reassigned replicas
* OAR = Original list of replicas for partition
* AR = current assigned replicas
* RS = current assigned replica set
* ORS = Original replica set for partition
* TRS = Reassigned (target) replica set
* AR = The replicas we are adding as part of this reassignment
* RR = The replicas we are removing as part of this reassignment
*
* 1. Update AR in ZK with OAR + RAR.
* 2. Send LeaderAndIsr request to every replica in OAR + RAR (with AR as OAR + RAR). We do this by forcing an update
* of the leader epoch in zookeeper.
* 3. Start new replicas RAR - OAR by moving replicas in RAR - OAR to NewReplica state.
* 4. Wait until all replicas in RAR are in sync with the leader.
* 5 Move all replicas in RAR to OnlineReplica state.
* 6. Set AR to RAR in memory.
* 7. If the leader is not in RAR, elect a new leader from RAR. If new leader needs to be elected from RAR, a LeaderAndIsr
* will be sent. If not, then leader epoch will be incremented in zookeeper and a LeaderAndIsr request will be sent.
* In any case, the LeaderAndIsr request will have AR = RAR. This will prevent the leader from adding any replica in
* RAR - OAR back in the isr.
* 8. Move all replicas in OAR - RAR to OfflineReplica state. As part of OfflineReplica state change, we shrink the
* isr to remove OAR - RAR in zookeeper and send a LeaderAndIsr ONLY to the Leader to notify it of the shrunk isr.
* After that, we send a StopReplica (delete = false) to the replicas in OAR - RAR.
* 9. Move all replicas in OAR - RAR to NonExistentReplica state. This will send a StopReplica (delete = true) to
* the replicas in OAR - RAR to physically delete the replicas on disk.
* 10. Update AR in ZK with RAR.
* 11. Update the /admin/reassign_partitions path in ZK to remove this partition.
* 12. After electing leader, the replicas and isr information changes. So resend the update metadata request to every broker.
* A reassignment may have up to three phases, each with its own steps:
*
* For example, if OAR = {1, 2, 3} and RAR = {4,5,6}, the values in the assigned replica (AR) and leader/isr path in ZK
* may go through the following transition.
* AR leader/isr
* {1,2,3} 1/{1,2,3} (initial state)
* {1,2,3,4,5,6} 1/{1,2,3} (step 2)
* {1,2,3,4,5,6} 1/{1,2,3,4,5,6} (step 4)
* {1,2,3,4,5,6} 4/{1,2,3,4,5,6} (step 7)
* {1,2,3,4,5,6} 4/{4,5,6} (step 8)
* {4,5,6} 4/{4,5,6} (step 10)
*
* Note that we have to update AR in ZK with RAR last since it's the only place where we store OAR persistently.
* Cleanup Phase: In the cases where this reassignment has to override an ongoing reassignment.
* . The ongoing reassignment is in phase A
* . ORS denotes the original replica set, prior to the ongoing reassignment
* . URS denotes the unnecessary replicas, ones which are currently part of the AR of the ongoing reassignment but will not be part of the new one
* . OVRS denotes the overlapping replica set - replicas which are part of the AR of the ongoing reassignment and will be part of the overriding reassignment
* (it is essentially (RS - ORS) - URS)
*
* 1 Set RS = ORS + OVRS, AR = OVRS, RS = [] in memory
* 2 Send LeaderAndIsr request with RS = ORS + OVRS, AR = [], RS = [] to all brokers in ORS + OVRS
* (because the ongoing reassignment is in phase A, we know we wouldn't have a leader in URS
* unless a preferred leader election was triggered while the reassignment was happening)
* 3 Replicas in URS -> Offline (force those replicas out of ISR)
* 4 Replicas in URS -> NonExistentReplica (force those replicas to be deleted)
*
* Phase A: Initial trigger (when TRS != ISR)
* A1. Update ZK with RS = ORS + TRS,
* AR = TRS - ORS and
* RR = ORS - TRS.
* A2. Update memory with RS = ORS + TRS, AR = TRS - ORS and RR = ORS - TRS
* A3. Send LeaderAndIsr request to every replica in ORS + TRS (with the new RS, AR and RR).
* We do this by forcing an update of the leader epoch in zookeeper.
* A4. Start new replicas AR by moving replicas in AR to NewReplica state.
*
* Phase B: All of TRS have caught up with the leaders and are in ISR
* B1. Move all replicas in TRS to OnlineReplica state.
* B2. Set RS = TRS, AR = [], RR = [] in memory.
* B3. Send a LeaderAndIsr request with RS = TRS. This will prevent the leader from adding any replica in TRS - ORS back in the isr.
* If the current leader is not in TRS or isn't alive, we move the leader to a new replica in TRS.
* We may send the LeaderAndIsr to more than the TRS replicas due to the
* way the partition state machine works (it reads replicas from ZK)
* B4. Move all replicas in RR to OfflineReplica state. As part of OfflineReplica state change, we shrink the
* isr to remove RR in ZooKeeper and send a LeaderAndIsr ONLY to the Leader to notify it of the shrunk isr.
* After that, we send a StopReplica (delete = false) to the replicas in RR.
* B5. Move all replicas in RR to NonExistentReplica state. This will send a StopReplica (delete = true) to
* the replicas in RR to physically delete the replicas on disk.
* B6. Update ZK with RS=TRS, AR=[], RR=[].
* B7. Remove the ISR reassign listener and maybe update the /admin/reassign_partitions path in ZK to remove this partition from it if present.
* B8. After electing leader, the replicas and isr information changes. So resend the update metadata request to every broker.
*
* In general, there are two goals we want to aim for:
* 1. Every replica present in the replica set of a LeaderAndIsrRequest gets the request sent to it
* 2. Replicas that are removed from a partition's assignment get StopReplica sent to them
*
* For example, if ORS = {1,2,3} and TRS = {4,5,6}, the values in the topic and leader/isr paths in ZK
* may go through the following transitions.
* RS AR RR leader isr
* {1,2,3} {} {} 1 {1,2,3} (initial state)
* {4,5,6,1,2,3} {4,5,6} {1,2,3} 1 {1,2,3} (step A2)
* {4,5,6,1,2,3} {4,5,6} {1,2,3} 1 {1,2,3,4,5,6} (phase B)
* {4,5,6,1,2,3} {4,5,6} {1,2,3} 4 {1,2,3,4,5,6} (step B3)
* {4,5,6,1,2,3} {4,5,6} {1,2,3} 4 {4,5,6} (step B4)
* {4,5,6} {} {} 4 {4,5,6} (step B6)
*
* Note that we have to update RS in ZK with TRS last since it's the only place where we store ORS persistently.
* This way, if the controller crashes before that step, we can still recover.
*/
private def onPartitionReassignment(topicPartition: TopicPartition, reassignedPartitionContext: ReassignedPartitionsContext): Unit = {
val reassignedReplicas = reassignedPartitionContext.newReplicas
if (!areReplicasInIsr(topicPartition, reassignedReplicas)) {
info(s"New replicas ${reassignedReplicas.mkString(",")} for partition $topicPartition being reassigned not yet " +
"caught up with the leader")
val newReplicasNotInOldReplicaList = reassignedReplicas.toSet -- controllerContext.partitionReplicaAssignment(topicPartition).toSet
val newAndOldReplicas = (reassignedPartitionContext.newReplicas ++ controllerContext.partitionReplicaAssignment(topicPartition)).toSet
//1. Update AR in ZK with OAR + RAR.
updateAssignedReplicasForPartition(topicPartition, newAndOldReplicas.toSeq)
//2. Send LeaderAndIsr request to every replica in OAR + RAR (with AR as OAR + RAR).
updateLeaderEpochAndSendRequest(topicPartition, controllerContext.partitionReplicaAssignment(topicPartition),
newAndOldReplicas.toSeq)
//3. replicas in RAR - OAR -> NewReplica
startNewReplicasForReassignedPartition(topicPartition, reassignedPartitionContext, newReplicasNotInOldReplicaList)
info(s"Waiting for new replicas ${reassignedReplicas.mkString(",")} for partition ${topicPartition} being " +
"reassigned to catch up with the leader")
} else {
//4. Wait until all replicas in RAR are in sync with the leader.
val oldReplicas = controllerContext.partitionReplicaAssignment(topicPartition).toSet -- reassignedReplicas.toSet
//5. replicas in RAR -> OnlineReplica
reassignedReplicas.foreach { replica =>
replicaStateMachine.handleStateChanges(Seq(new PartitionAndReplica(topicPartition, replica)), OnlineReplica)
val originalAssignmentOpt = maybeRevertOngoingReassignment(topicPartition, reassignedPartitionContext)
val oldReplicas = originalAssignmentOpt match {
case Some(originalReplicas) => originalReplicas
case None => controllerContext.partitionFullReplicaAssignment(topicPartition).previousAssignment.replicas
}
//6. Set AR to RAR in memory.
//7. Send LeaderAndIsr request with a potential new leader (if current leader not in RAR) and
// a new AR (using RAR) and same isr to every broker in RAR
moveReassignedPartitionLeaderIfRequired(topicPartition, reassignedPartitionContext)
//8. replicas in OAR - RAR -> Offline (force those replicas out of isr)
//9. replicas in OAR - RAR -> NonExistentReplica (force those replicas to be deleted)
stopOldReplicasOfReassignedPartition(topicPartition, reassignedPartitionContext, oldReplicas)
//10. Update AR in ZK with RAR.
updateAssignedReplicasForPartition(topicPartition, reassignedReplicas)
//11. Update the /admin/reassign_partitions path in ZK to remove this partition.
removePartitionsFromReassignedPartitions(Set(topicPartition))
//12. After electing leader, the replicas and isr information changes, so resend the update metadata request to every broker
// RS = ORS + TRS, AR = TRS - ORS and RR = ORS - TRS
val partitionAssignment = PartitionReplicaAssignment.fromOldAndNewReplicas(
oldReplicas = oldReplicas,
newReplicas = reassignedPartitionContext.newReplicas)
assert(reassignedPartitionContext.newReplicas == partitionAssignment.targetReplicas,
s"newReplicas ${reassignedPartitionContext.newReplicas} were not equal to the expected targetReplicas ${partitionAssignment.targetReplicas}")
val targetReplicas = partitionAssignment.targetReplicas
if (!areReplicasInIsr(topicPartition, targetReplicas)) {
info(s"New replicas ${targetReplicas.mkString(",")} for partition $topicPartition being reassigned not yet " +
"caught up with the leader")
// A1. Update ZK with RS = ORS + TRS, AR = TRS - ORS and RR = ORS - TRS.
updateReplicaAssignmentForPartition(topicPartition, partitionAssignment)
// A2. Update memory with RS = ORS + TRS, AR = TRS - ORS and RR = ORS - TRS
controllerContext.updatePartitionFullReplicaAssignment(topicPartition, partitionAssignment)
// A3. Send LeaderAndIsr request to every replica in ORS + TRS (with the new RS, AR and RR).
val updatedAssignment = controllerContext.partitionFullReplicaAssignment(topicPartition)
updateLeaderEpochAndSendRequest(topicPartition, updatedAssignment.replicas, updatedAssignment)
// A4. replicas in AR -> NewReplica
startNewReplicasForReassignedPartition(topicPartition, updatedAssignment.addingReplicas)
info(s"Waiting for new replicas ${updatedAssignment.addingReplicas.mkString(",")} for partition $topicPartition being " +
s"reassigned to catch up with the leader (target replicas ${updatedAssignment.targetReplicas})")
} else {
// B1. replicas in TRS -> OnlineReplica
targetReplicas.foreach { replica =>
replicaStateMachine.handleStateChanges(Seq(PartitionAndReplica(topicPartition, replica)), OnlineReplica)
}
// B2. Set RS = TRS, AR = [], RR = [] in memory.
// B3. Send LeaderAndIsr request with a potential new leader (if current leader not in TRS) and
// a new RS (using TRS) and same isr to every broker in ORS + TRS or TRS
moveReassignedPartitionLeaderIfRequired(topicPartition, reassignedPartitionContext, partitionAssignment)
// B4. replicas in RR -> Offline (force those replicas out of isr)
// B5. replicas in RR -> NonExistentReplica (force those replicas to be deleted)
stopRemovedReplicasOfReassignedPartition(topicPartition, partitionAssignment.removingReplicas)
// B6. Update ZK with RS = TRS, AR = [], RR = [].
updateReplicaAssignmentForPartition(topicPartition, controllerContext.partitionFullReplicaAssignment(topicPartition))
// B7. Remove the ISR reassign listener and maybe update the /admin/reassign_partitions path in ZK to remove this partition from it.
removePartitionFromReassignedPartitions(topicPartition)
// B8. After electing a leader in B3, the replicas and isr information changes, so resend the update metadata request to every broker
sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq, Set(topicPartition))
// signal delete topic thread if reassignment for some partitions belonging to topics being deleted just completed
topicDeletionManager.resumeDeletionForTopics(Set(topicPartition.topic))
}
}
/**
* This is called in case we need to override/revert an ongoing reassignment.
* Note that due to the way we compute the original replica set, we have no guarantee that a revert would put it in the same order.
* @return An option of the original replicas prior to the ongoing reassignment. None if there is no ongoing reassignment
*/
private def maybeRevertOngoingReassignment(topicPartition: TopicPartition, reassignedPartitionContext: ReassignedPartitionsContext): Option[Seq[Int]] = {
reassignedPartitionContext.ongoingReassignmentOpt match {
case Some(ongoingAssignment) =>
val originalAssignment = ongoingAssignment.previousAssignment
assert(ongoingAssignment.isBeingReassigned)
assert(!originalAssignment.isBeingReassigned)
val unnecessaryReplicas = ongoingAssignment.replicas.filterNot(originalAssignment.replicas.contains(_))
val (overlappingReplicas, replicasToRemove) = unnecessaryReplicas.partition(reassignedPartitionContext.newReplicas.contains(_))
// RS = ORS + OVRS, AR = OVRS, RR = []
val intermediateReplicaAssignment = PartitionReplicaAssignment(originalAssignment.replicas ++ overlappingReplicas, overlappingReplicas, Seq())
if (isDebugEnabled)
debug(s"Reverting previous reassignment $originalAssignment (we were in the " +
s"process of an ongoing reassignment with target replicas ${ongoingAssignment.targetReplicas.mkString(",")} (" +
s"Overlapping replicas: ${overlappingReplicas.mkString(",")}, Replicas to remove: ${replicasToRemove.mkString(",")})")
// Set RS = ORS + OVRS, AR = OVRS, RR = [] in memory.
controllerContext.updatePartitionFullReplicaAssignment(topicPartition, intermediateReplicaAssignment)
// Increment leader epoch and send LeaderAndIsr with new RS (using old replicas and overlapping replicas) and same ISR to every broker in ORS + OVRS
// This will prevent the leader from adding any replica outside RS back in the ISR
updateLeaderEpochAndSendRequest(topicPartition, intermediateReplicaAssignment.replicas, intermediateReplicaAssignment)
// replicas in URS -> Offline (force those replicas out of isr)
// replicas in URS -> NonExistentReplica (force those replicas to be deleted)
stopRemovedReplicasOfReassignedPartition(topicPartition, replicasToRemove)
reassignedPartitionContext.ongoingReassignmentOpt = None
Some(originalAssignment.replicas)
case None => None // nothing to revert
}
}
/**
* Trigger partition reassignment for the provided partitions if the assigned replicas are not the same as the
* reassigned replicas (as defined in `ControllerContext.partitionsBeingReassigned`) and if the topic has not been
* deleted.
*
* Called when:
* 1. zNode is first created
* 2. Controller fail over
* 3. AlterPartitionReassignments API is called
*
* `partitionsBeingReassigned` must be populated with all partitions being reassigned before this method is invoked
* as explained in the method documentation of `removePartitionFromReassignedPartitions` (which is invoked by this
* method).
*
* @throws IllegalStateException if a partition is not in `partitionsBeingReassigned`
*/
private def maybeTriggerPartitionReassignment(topicPartitions: Set[TopicPartition]): Unit = {
private def maybeTriggerPartitionReassignment(topicPartitions: Set[TopicPartition]): Map[TopicPartition, ApiError] = {
val reassignmentResults: mutable.Map[TopicPartition, ApiError] = mutable.Map.empty
val partitionsToBeRemovedFromReassignment = scala.collection.mutable.Set.empty[TopicPartition]
topicPartitions.foreach { tp =>
if (topicDeletionManager.isTopicQueuedUpForDeletion(tp.topic)) {
info(s"Skipping reassignment of $tp since the topic is currently being deleted")
partitionsToBeRemovedFromReassignment.add(tp)
reassignmentResults.put(tp, new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, "The partition does not exist."))
} else {
val reassignedPartitionContext = controllerContext.partitionsBeingReassigned.get(tp).getOrElse {
throw new IllegalStateException(s"Initiating reassign replicas for partition $tp not present in " +
@ -645,32 +738,42 @@ class KafkaController(val config: KafkaConfig,
info(s"Partition $tp to be reassigned is already assigned to replicas " +
s"${newReplicas.mkString(",")}. Ignoring request for partition reassignment.")
partitionsToBeRemovedFromReassignment.add(tp)
reassignmentResults.put(tp, ApiError.NONE)
} else {
try {
info(s"Handling reassignment of partition $tp to new replicas ${newReplicas.mkString(",")}")
info(s"Handling reassignment of partition $tp from current replicas ${assignedReplicas.mkString(",")}" +
s"to new replicas ${newReplicas.mkString(",")}")
// first register ISR change listener
reassignedPartitionContext.registerReassignIsrChangeHandler(zkClient)
// mark topic ineligible for deletion for the partitions being reassigned
topicDeletionManager.markTopicIneligibleForDeletion(Set(topic),
reason = "topic reassignment in progress")
onPartitionReassignment(tp, reassignedPartitionContext)
reassignmentResults.put(tp, ApiError.NONE)
} catch {
case e: ControllerMovedException =>
error(s"Error completing reassignment of partition $tp because controller has moved to another broker", e)
throw e
case e: Throwable =>
error(s"Error completing reassignment of partition $tp", e)
// remove the partition from the admin path to unblock the admin client
partitionsToBeRemovedFromReassignment.add(tp)
zkClient.getFullReplicaAssignmentForTopics(immutable.Set(tp.topic())).find(_._1 == tp) match {
case Some(persistedAssignment) =>
controllerContext.updatePartitionFullReplicaAssignment(tp, persistedAssignment._2)
case None =>
}
reassignmentResults.put(tp, new ApiError(Errors.UNKNOWN_SERVER_ERROR))
}
}
} else {
error(s"Ignoring request to reassign partition $tp that doesn't exist.")
partitionsToBeRemovedFromReassignment.add(tp)
reassignmentResults.put(tp, new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, "The partition does not exist."))
}
}
}
removePartitionsFromReassignedPartitions(partitionsToBeRemovedFromReassignment)
reassignmentResults
}
/**
@ -730,8 +833,14 @@ class KafkaController(val config: KafkaConfig,
info(s"Initialized broker epochs cache: ${controllerContext.liveBrokerIdAndEpochs}")
controllerContext.allTopics = zkClient.getAllTopicsInCluster.toSet
registerPartitionModificationsHandlers(controllerContext.allTopics.toSeq)
zkClient.getReplicaAssignmentForTopics(controllerContext.allTopics.toSet).foreach {
case (topicPartition, assignedReplicas) => controllerContext.updatePartitionReplicaAssignment(topicPartition, assignedReplicas)
zkClient.getFullReplicaAssignmentForTopics(controllerContext.allTopics.toSet).foreach {
case (topicPartition, replicaAssignment) =>
controllerContext.updatePartitionFullReplicaAssignment(topicPartition, replicaAssignment)
if (replicaAssignment.isBeingReassigned) {
val reassignIsrChangeHandler = new PartitionReassignmentIsrChangeHandler(eventManager, topicPartition)
controllerContext.partitionsBeingReassigned.put(topicPartition, ReassignedPartitionsContext(replicaAssignment.targetReplicas,
reassignIsrChangeHandler, persistedInZk = false, ongoingReassignmentOpt = None))
}
}
controllerContext.partitionLeadershipInfo.clear()
controllerContext.shuttingDownBrokerIds = mutable.Set.empty[Int]
@ -767,14 +876,27 @@ class KafkaController(val config: KafkaConfig,
pendingPreferredReplicaElections
}
/**
* Initializes the partitions being reassigned by reading them from the /admin/reassign_partitions znode
* This will overwrite any reassignments that were set by the AlterPartitionReassignments API
*/
private def initializePartitionReassignment(): Unit = {
// read the partitions being reassigned from zookeeper path /admin/reassign_partitions
val partitionsBeingReassigned = zkClient.getPartitionReassignment
info(s"Partitions being reassigned: $partitionsBeingReassigned")
info(s"DEPRECATED: Partitions being reassigned through ZooKeeper: $partitionsBeingReassigned")
controllerContext.partitionsBeingReassigned ++= partitionsBeingReassigned.iterator.map { case (tp, newReplicas) =>
partitionsBeingReassigned.foreach {
case (tp, newReplicas) =>
val reassignIsrChangeHandler = new PartitionReassignmentIsrChangeHandler(eventManager, tp)
tp -> ReassignedPartitionsContext(newReplicas, reassignIsrChangeHandler)
val assignment = controllerContext.partitionFullReplicaAssignment(tp)
val ongoingReassignmentOption = if (assignment.isBeingReassigned)
Some(assignment)
else
None
controllerContext.partitionsBeingReassigned += (
tp -> ReassignedPartitionsContext(newReplicas, reassignIsrChangeHandler,
persistedInZk = true,
ongoingReassignmentOpt = ongoingReassignmentOption))
}
}
@ -805,13 +927,19 @@ class KafkaController(val config: KafkaConfig,
}
private def moveReassignedPartitionLeaderIfRequired(topicPartition: TopicPartition,
reassignedPartitionContext: ReassignedPartitionsContext): Unit = {
reassignedPartitionContext: ReassignedPartitionsContext,
currentAssignment: PartitionReplicaAssignment): Unit = {
val reassignedReplicas = reassignedPartitionContext.newReplicas
val currentLeader = controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader
// change the assigned replica list to just the reassigned replicas in the cache so it gets sent out on the LeaderAndIsr
// request to the current or new leader. This will prevent it from adding the old replicas to the ISR
val oldAndNewReplicas = controllerContext.partitionReplicaAssignment(topicPartition)
controllerContext.updatePartitionReplicaAssignment(topicPartition, reassignedReplicas)
val newAssignment = PartitionReplicaAssignment(replicas = reassignedReplicas, addingReplicas = Seq(), removingReplicas = Seq())
controllerContext.updatePartitionFullReplicaAssignment(
topicPartition,
newAssignment
)
if (!reassignedPartitionContext.newReplicas.contains(currentLeader)) {
info(s"Leader $currentLeader for partition $topicPartition being reassigned, " +
s"is not in the new list of replicas ${reassignedReplicas.mkString(",")}. Re-electing leader")
@ -823,7 +951,7 @@ class KafkaController(val config: KafkaConfig,
info(s"Leader $currentLeader for partition $topicPartition being reassigned, " +
s"is already in the new list of replicas ${reassignedReplicas.mkString(",")} and is alive")
// shrink replication factor and update the leader epoch in zookeeper to use on the next LeaderAndIsrRequest
updateLeaderEpochAndSendRequest(topicPartition, oldAndNewReplicas, reassignedReplicas)
updateLeaderEpochAndSendRequest(topicPartition, newAssignment.replicas, newAssignment)
} else {
info(s"Leader $currentLeader for partition $topicPartition being reassigned, " +
s"is already in the new list of replicas ${reassignedReplicas.mkString(",")} but is dead")
@ -832,45 +960,46 @@ class KafkaController(val config: KafkaConfig,
}
}
private def stopOldReplicasOfReassignedPartition(topicPartition: TopicPartition,
reassignedPartitionContext: ReassignedPartitionsContext,
oldReplicas: Set[Int]): Unit = {
private def stopRemovedReplicasOfReassignedPartition(topicPartition: TopicPartition,
removedReplicas: Seq[Int]): Unit = {
// first move the replica to offline state (the controller removes it from the ISR)
val replicasToBeDeleted = oldReplicas.map(PartitionAndReplica(topicPartition, _))
replicaStateMachine.handleStateChanges(replicasToBeDeleted.toSeq, OfflineReplica)
val replicasToBeDeleted = removedReplicas.map(PartitionAndReplica(topicPartition, _))
replicaStateMachine.handleStateChanges(replicasToBeDeleted, OfflineReplica)
// send stop replica command to the old replicas
replicaStateMachine.handleStateChanges(replicasToBeDeleted.toSeq, ReplicaDeletionStarted)
replicaStateMachine.handleStateChanges(replicasToBeDeleted, ReplicaDeletionStarted)
// TODO: Eventually partition reassignment could use a callback that does retries if deletion failed
replicaStateMachine.handleStateChanges(replicasToBeDeleted.toSeq, ReplicaDeletionSuccessful)
replicaStateMachine.handleStateChanges(replicasToBeDeleted.toSeq, NonExistentReplica)
replicaStateMachine.handleStateChanges(replicasToBeDeleted, ReplicaDeletionSuccessful)
replicaStateMachine.handleStateChanges(replicasToBeDeleted, NonExistentReplica)
}
private def updateAssignedReplicasForPartition(partition: TopicPartition,
replicas: Seq[Int]): Unit = {
controllerContext.updatePartitionReplicaAssignment(partition, replicas)
val setDataResponse = zkClient.setTopicAssignmentRaw(partition.topic, controllerContext.partitionReplicaAssignmentForTopic(partition.topic), controllerContext.epochZkVersion)
private def updateReplicaAssignmentForPartition(partition: TopicPartition,
assignment: PartitionReplicaAssignment): Unit = {
var topicAssignment = controllerContext.partitionFullReplicaAssignmentForTopic(partition.topic)
topicAssignment += partition -> assignment
val setDataResponse = zkClient.setTopicAssignmentRaw(partition.topic, topicAssignment, controllerContext.epochZkVersion)
setDataResponse.resultCode match {
case Code.OK =>
info(s"Updated assigned replicas for partition $partition being reassigned to ${replicas.mkString(",")}")
// update the assigned replica list after a successful zookeeper write
controllerContext.updatePartitionReplicaAssignment(partition, replicas)
info(s"Updated assigned replicas for partition $partition being reassigned to ${assignment.targetReplicas.mkString(",")}" +
s" (addingReplicas: ${assignment.addingReplicas.mkString(",")}, removingReplicas: ${assignment.removingReplicas.mkString(",")})")
case Code.NONODE => throw new IllegalStateException(s"Topic ${partition.topic} doesn't exist")
case _ => throw new KafkaException(setDataResponse.resultException.get)
}
}
private def startNewReplicasForReassignedPartition(topicPartition: TopicPartition,
reassignedPartitionContext: ReassignedPartitionsContext,
newReplicas: Set[Int]): Unit = {
private def startNewReplicasForReassignedPartition(topicPartition: TopicPartition, newReplicas: Seq[Int]): Unit = {
// send the start replica request to the brokers in the reassigned replicas list that are not in the assigned
// replicas list
newReplicas.foreach { replica =>
replicaStateMachine.handleStateChanges(Seq(new PartitionAndReplica(topicPartition, replica)), NewReplica)
replicaStateMachine.handleStateChanges(Seq(PartitionAndReplica(topicPartition, replica)), NewReplica)
}
}
private def updateLeaderEpochAndSendRequest(partition: TopicPartition, replicasToReceiveRequest: Seq[Int], newAssignedReplicas: Seq[Int]): Unit = {
private def updateLeaderEpochAndSendRequest(partition: TopicPartition, replicasToReceiveRequest: Seq[Int],
newAssignedReplicas: PartitionReplicaAssignment): Unit = {
val stateChangeLog = stateChangeLogger.withControllerEpoch(controllerContext.epoch)
val replicaSetStr = s"replica set ${newAssignedReplicas.replicas.mkString(",")} " +
s"(addingReplicas: ${newAssignedReplicas.addingReplicas.mkString(",")}, removingReplicas: ${newAssignedReplicas.removingReplicas.mkString(",")})"
updateLeaderEpoch(partition) match {
case Some(updatedLeaderIsrAndControllerEpoch) =>
try {
@ -882,12 +1011,12 @@ class KafkaController(val config: KafkaConfig,
case e: IllegalStateException =>
handleIllegalState(e)
}
stateChangeLog.trace(s"Sent LeaderAndIsr request $updatedLeaderIsrAndControllerEpoch with new assigned replica " +
s"list ${newAssignedReplicas.mkString(",")} to leader ${updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader} " +
stateChangeLog.trace(s"Sent LeaderAndIsr request $updatedLeaderIsrAndControllerEpoch with new assigned $replicaSetStr" +
s"to leader ${updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader} " +
s"for partition being reassigned $partition")
case None => // fail the reassignment
stateChangeLog.error("Failed to send LeaderAndIsr request with new assigned replica list " +
s"${newAssignedReplicas.mkString( ",")} to leader for partition being reassigned $partition")
stateChangeLog.error(s"Failed to send LeaderAndIsr request with new assigned $replicaSetStr " +
s"to leader for partition being reassigned $partition")
}
}
@ -914,7 +1043,7 @@ class KafkaController(val config: KafkaConfig,
* is complete (i.e. there is no other partition with a reassignment in progress), the reassign_partitions znode
* is deleted.
*
* `ControllerContext.partitionsBeingReassigned` must be populated with all partitions being reassigned before this
* `ControllerContext.partitionsBeingReassigned` must be populated with all the zk-persisted partition reassignments before this
* method is invoked to avoid premature deletion of the `reassign_partitions` znode.
*/
private def removePartitionsFromReassignedPartitions(partitionsToBeRemoved: Set[TopicPartition]): Unit = {
@ -922,7 +1051,33 @@ class KafkaController(val config: KafkaConfig,
reassignContext.unregisterReassignIsrChangeHandler(zkClient)
}
val updatedPartitionsBeingReassigned = controllerContext.partitionsBeingReassigned -- partitionsToBeRemoved
removePartitionsFromZkReassignment(partitionsToBeRemoved)
controllerContext.partitionsBeingReassigned --= partitionsToBeRemoved
}
private def removePartitionFromReassignedPartitions(partitionToBeRemoved: TopicPartition) {
controllerContext.partitionsBeingReassigned.get(partitionToBeRemoved) match {
case Some(reassignContext) =>
reassignContext.unregisterReassignIsrChangeHandler(zkClient)
if (reassignContext.persistedInZk) {
removePartitionsFromZkReassignment(Set(partitionToBeRemoved))
}
controllerContext.partitionsBeingReassigned.remove(partitionToBeRemoved)
case None =>
throw new IllegalStateException("Cannot remove a reassigning partition because it is not present in memory")
}
}
private def removePartitionsFromZkReassignment(partitionsToBeRemoved: Set[TopicPartition]): Unit = {
if (!zkClient.reassignPartitionsInProgress()) {
debug(s"Cannot remove partitions $partitionsToBeRemoved from ZooKeeper because there is no ZooKeeper reassignment present")
return
}
val updatedPartitionsBeingReassigned = controllerContext.partitionsBeingReassigned.filter(_._2.persistedInZk).toMap -- partitionsToBeRemoved
info(s"Removing partitions $partitionsToBeRemoved from the list of reassigned partitions in zookeeper")
@ -931,7 +1086,7 @@ class KafkaController(val config: KafkaConfig,
info(s"No more partitions need to be reassigned. Deleting zk path ${ReassignPartitionsZNode.path}")
zkClient.deletePartitionReassignment(controllerContext.epochZkVersion)
// Ensure we detect future reassignments
eventManager.put(PartitionReassignment)
eventManager.put(PartitionReassignment(None, None))
} else {
val reassignment = updatedPartitionsBeingReassigned.map { case (k, v) => k -> v.newReplicas }
try zkClient.setOrCreatePartitionReassignment(reassignment, controllerContext.epochZkVersion)
@ -939,8 +1094,6 @@ class KafkaController(val config: KafkaConfig,
case e: KeeperException => throw new AdminOperationException(e)
}
}
controllerContext.partitionsBeingReassigned --= partitionsToBeRemoved
}
private def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicPartition],
@ -1216,11 +1369,21 @@ class KafkaController(val config: KafkaConfig,
0
} else {
controllerContext.allPartitions.count { topicPartition =>
val replicas = controllerContext.partitionReplicaAssignment(topicPartition)
val replicaAssignment: PartitionReplicaAssignment = controllerContext.partitionFullReplicaAssignment(topicPartition)
val replicas = replicaAssignment.replicas
val preferredReplica = replicas.head
val leadershipInfo = controllerContext.partitionLeadershipInfo.get(topicPartition)
leadershipInfo.map(_.leaderAndIsr.leader != preferredReplica).getOrElse(false) &&
!topicDeletionManager.isTopicQueuedUpForDeletion(topicPartition.topic)
val isImbalanced = controllerContext.partitionLeadershipInfo.get(topicPartition) match {
case Some(leadershipInfo) =>
if (replicaAssignment.isBeingReassigned && replicaAssignment.addingReplicas.contains(preferredReplica))
// reassigning partitions are not counted as imbalanced until the new replica joins the ISR (completes reassignment)
leadershipInfo.leaderAndIsr.isr.contains(preferredReplica)
else
leadershipInfo.leaderAndIsr.leader != preferredReplica
case None => false
}
isImbalanced && !topicDeletionManager.isTopicQueuedUpForDeletion(topicPartition.topic)
}
}
@ -1388,10 +1551,10 @@ class KafkaController(val config: KafkaConfig,
controllerContext.allTopics = topics
registerPartitionModificationsHandlers(newTopics.toSeq)
val addedPartitionReplicaAssignment = zkClient.getReplicaAssignmentForTopics(newTopics)
val addedPartitionReplicaAssignment = zkClient.getFullReplicaAssignmentForTopics(newTopics)
deletedTopics.foreach(controllerContext.removeTopic)
addedPartitionReplicaAssignment.foreach {
case (topicAndPartition, newReplicas) => controllerContext.updatePartitionReplicaAssignment(topicAndPartition, newReplicas)
case (topicAndPartition, newReplicaAssignment) => controllerContext.updatePartitionFullReplicaAssignment(topicAndPartition, newReplicaAssignment)
}
info(s"New topics: [$newTopics], deleted topics: [$deletedTopics], new partition replica assignment " +
s"[$addedPartitionReplicaAssignment]")
@ -1416,10 +1579,15 @@ class KafkaController(val config: KafkaConfig,
info("Restoring the partition replica assignment for topic %s".format(topic))
val existingPartitions = zkClient.getChildren(TopicPartitionsZNode.path(topic))
val existingPartitionReplicaAssignment = newPartitionReplicaAssignment.filter(p =>
existingPartitions.contains(p._1.partition.toString))
val existingPartitionReplicaAssignment = newPartitionReplicaAssignment
.filter(p => existingPartitions.contains(p._1.partition.toString))
.map { case (tp, _) =>
tp -> controllerContext.partitionFullReplicaAssignment(tp)
}.toMap
zkClient.setTopicAssignment(topic, existingPartitionReplicaAssignment, controllerContext.epochZkVersion)
zkClient.setTopicAssignment(topic,
existingPartitionReplicaAssignment,
controllerContext.epochZkVersion)
}
if (!isActive) return
@ -1479,25 +1647,134 @@ class KafkaController(val config: KafkaConfig,
}
}
private def processPartitionReassignment(): Unit = {
if (!isActive) return
/**
* Process a partition reassignment.
* A partition reassignment can be triggered through the AlterPartitionReassignment API (in which case reassignmentsOpt is present)
* or through the /admin/reassign_partitions znode (in which case reassignmentsOpt is None).
* In both cases, we need to populate `partitionsBeingReassigned` with all partitions being reassigned
* before invoking `maybeTriggerPartitionReassignment` (see method documentation for the reason)
*
* @param reassignmentsOpt - optional map of reassignments, expected when an API reassignment is issued
* The map consists of topic partitions to an optional sequence of target replicas.
* An empty target replicas option denotes a revert of an on-going reassignment.
* @param callback - optional callback, expected when an API reassignment is issued
*/
private def processPartitionReassignment(reassignmentsOpt: Option[Map[TopicPartition, Option[Seq[Int]]]],
callback: Option[AlterReassignmentsCallback]): Unit = {
if (!isActive) {
callback match {
case Some(cb) => cb(Right(new ApiError(Errors.NOT_CONTROLLER)))
case None =>
}
return
}
val reassignmentResults: mutable.Map[TopicPartition, ApiError] = mutable.Map.empty
val partitionsToBeReassigned = reassignmentsOpt match {
case Some(reassignments) => // API triggered
val (savedReassignments, _) = reassignments.partition { case (tp, targetReplicas) =>
if (replicasAreValid(targetReplicas)) {
savePartitionReassignment(reassignmentResults, tp, targetReplicas, zkTriggered = false)
} else {
reassignmentResults.put(tp, new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, "The partition assignment is not valid."))
false
}
}
savedReassignments.keySet
case None => // ZK triggered
// We need to register the watcher if the path doesn't exist in order to detect future reassignments and we get
// the `path exists` check for free
if (zkClient.registerZNodeChangeHandlerAndCheckExistence(partitionReassignmentHandler)) {
val partitionReassignment = zkClient.getPartitionReassignment
// Populate `partitionsBeingReassigned` with all partitions being reassigned before invoking
// `maybeTriggerPartitionReassignment` (see method documentation for the reason)
partitionReassignment.foreach { case (tp, newReplicas) =>
val reassignIsrChangeHandler = new PartitionReassignmentIsrChangeHandler(eventManager, tp)
controllerContext.partitionsBeingReassigned.put(tp, ReassignedPartitionsContext(newReplicas, reassignIsrChangeHandler))
val (savedReassignments, _) = partitionReassignment.partition { case (tp, targetReplicas) =>
savePartitionReassignment(reassignmentResults, tp, Some(targetReplicas), zkTriggered = true)
}
savedReassignments.keySet
} else {
Set.empty[TopicPartition]
}
}
maybeTriggerPartitionReassignment(partitionReassignment.keySet)
reassignmentResults ++= maybeTriggerPartitionReassignment(partitionsToBeReassigned)
callback match {
case Some(cb) => cb(Left(reassignmentResults))
case None =>
}
}
private def replicasAreValid(replicasOpt: Option[Seq[Int]]): Boolean = {
replicasOpt match {
case Some(replicas) =>
val replicaSet = replicas.toSet
if (replicas.size != replicaSet.size)
false
else if (replicas.exists(_ < 0))
false
else
replicaSet.subsetOf(controllerContext.liveBrokerIds)
case None => true
}
}
private def savePartitionReassignment(reassignmentResults: mutable.Map[TopicPartition, ApiError], partition: TopicPartition,
targetReplicasOpt: Option[Seq[Int]], zkTriggered: Boolean): Boolean = {
val reassignIsrChangeHandler = new PartitionReassignmentIsrChangeHandler(eventManager, partition)
val replicaAssignment = controllerContext.partitionFullReplicaAssignment(partition)
val reassignmentIsInProgress = controllerContext.partitionsBeingReassigned.contains(partition)
val newContextOpt = targetReplicasOpt match {
case Some(targetReplicas) =>
if (reassignmentIsInProgress) {
info(s"Overriding old reassignment for partition $partition " +
s"(with target replicas ${replicaAssignment.targetReplicas.mkString(",")}) " +
s"to new target replicas (${targetReplicas.mkString(",")})")
assert(replicaAssignment.isBeingReassigned)
val oldContext = controllerContext.partitionsBeingReassigned(partition)
oldContext.unregisterReassignIsrChangeHandler(zkClient)
Some(ReassignedPartitionsContext(targetReplicas, reassignIsrChangeHandler,
persistedInZk = zkTriggered || oldContext.persistedInZk,
ongoingReassignmentOpt = Some(replicaAssignment))
)
} else {
Some(ReassignedPartitionsContext(targetReplicas, reassignIsrChangeHandler,
persistedInZk = zkTriggered,
ongoingReassignmentOpt = None)
)
}
case None => // revert
if (reassignmentIsInProgress) {
val originalAssignment = replicaAssignment.previousAssignment
info(s"Reverting old reassignment for partition $partition " +
s"(with target replicas ${replicaAssignment.targetReplicas.mkString(",")}) " +
s"to original replicas (${originalAssignment.replicas.mkString(",")})")
assert(replicaAssignment.isBeingReassigned)
val oldContext = controllerContext.partitionsBeingReassigned(partition)
oldContext.unregisterReassignIsrChangeHandler(zkClient)
Some(ReassignedPartitionsContext(originalAssignment.replicas, reassignIsrChangeHandler,
persistedInZk = oldContext.persistedInZk,
ongoingReassignmentOpt = Some(replicaAssignment)
))
} else {
reassignmentResults.put(partition, new ApiError(Errors.NO_REASSIGNMENT_IN_PROGRESS))
None
}
}
newContextOpt match {
case Some(newContext) =>
controllerContext.partitionsBeingReassigned.put(partition, newContext)
true
case None => false
}
}
private def processPartitionReassignmentIsrChange(partition: TopicPartition): Unit = {
if (!isActive) return
// check if this partition is still being reassigned or not
@ -1524,6 +1801,30 @@ class KafkaController(val config: KafkaConfig,
}
}
private def processListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]], callback: ListReassignmentsCallback): Unit = {
if (!isActive) {
callback(Right(new ApiError(Errors.NOT_CONTROLLER)))
} else {
val results: mutable.Map[TopicPartition, PartitionReplicaAssignment] = mutable.Map.empty
val partitionsToList = partitionsOpt match {
case Some(partitions) => partitions
case None => controllerContext.partitionsBeingReassigned.keys
}
partitionsToList.foreach { tp =>
val assignment = controllerContext.partitionFullReplicaAssignment(tp)
if (assignment.replicas.isEmpty) {
callback(Right(new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION)))
return
} else if (assignment.isBeingReassigned) {
results += tp -> assignment
}
}
callback(Left(results))
}
}
private def processIsrChangeNotification(): Unit = {
def processUpdateNotifications(partitions: Seq[TopicPartition]): Unit = {
val liveBrokers: Seq[Int] = controllerContext.liveOrShuttingDownBrokerIds.toSeq
@ -1553,6 +1854,16 @@ class KafkaController(val config: KafkaConfig,
eventManager.put(ReplicaLeaderElection(Some(partitions), electionType, AdminClientTriggered, callback))
}
def listPartitionReassignments(partitions: Option[Set[TopicPartition]],
callback: ListReassignmentsCallback): Unit = {
eventManager.put(ListPartitionReassignments(partitions, callback))
}
def alterPartitionReassignments(partitions: Map[TopicPartition, Option[Seq[Int]]],
callback: AlterReassignmentsCallback): Unit = {
eventManager.put(PartitionReassignment(Some(partitions), Some(callback)))
}
private def preemptReplicaLeaderElection(
partitionsFromAdminClientOpt: Option[Set[TopicPartition]],
callback: ElectLeadersCallback
@ -1700,8 +2011,10 @@ class KafkaController(val config: KafkaConfig,
processPartitionModifications(topic)
case TopicDeletion =>
processTopicDeletion()
case PartitionReassignment =>
processPartitionReassignment()
case PartitionReassignment(reassignments, callback) =>
processPartitionReassignment(reassignments, callback)
case ListPartitionReassignments(partitions, callback) =>
processListPartitionReassignments(partitions, callback)
case PartitionReassignmentIsrChange(partition) =>
processPartitionReassignmentIsrChange(partition)
case IsrChangeNotification =>
@ -1781,7 +2094,7 @@ class PartitionReassignmentHandler(eventManager: ControllerEventManager) extends
// Note that the event is also enqueued when the znode is deleted, but we do it explicitly instead of relying on
// handleDeletion(). This approach is more robust as it doesn't depend on the watcher being re-registered after
// it's consumed during data changes (we ensure re-registration when the znode is deleted).
override def handleCreation(): Unit = eventManager.put(PartitionReassignment)
override def handleCreation(): Unit = eventManager.put(PartitionReassignment(None, None))
}
class PartitionReassignmentIsrChangeHandler(eventManager: ControllerEventManager, partition: TopicPartition) extends ZNodeChangeHandler {
@ -1814,8 +2127,17 @@ class ControllerChangeHandler(eventManager: ControllerEventManager) extends ZNod
override def handleDataChange(): Unit = eventManager.put(ControllerChange)
}
/**
* @param newReplicas - the target replicas for this partition
* @param reassignIsrChangeHandler - a handler for tracking ISR changes in this partition
* @param persistedInZk - a boolean indicating whether this partition is stored in the /admin/reassign_partitions znode
* it is needed to ensure that an API reassignment that overrides a znode reassignment still cleans up after itself
* @param ongoingReassignmentOpt - the ongoing reassignment for this partition, if one is present -- it will be reverted.
*/
case class ReassignedPartitionsContext(var newReplicas: Seq[Int] = Seq.empty,
reassignIsrChangeHandler: PartitionReassignmentIsrChangeHandler) {
reassignIsrChangeHandler: PartitionReassignmentIsrChangeHandler,
persistedInZk: Boolean,
var ongoingReassignmentOpt: Option[PartitionReplicaAssignment]) {
def registerReassignIsrChangeHandler(zkClient: KafkaZkClient): Unit =
zkClient.registerZNodeChangeHandler(reassignIsrChangeHandler)
@ -1934,12 +2256,13 @@ case object TopicDeletion extends ControllerEvent {
override def state: ControllerState = ControllerState.TopicDeletion
}
case object PartitionReassignment extends ControllerEvent {
override def state: ControllerState = ControllerState.PartitionReassignment
case class PartitionReassignment(reassignments: Option[Map[TopicPartition, Option[Seq[Int]]]],
callback: Option[AlterReassignmentsCallback]) extends ControllerEvent {
override def state: ControllerState = ControllerState.AlterPartitionReassignment
}
case class PartitionReassignmentIsrChange(partition: TopicPartition) extends ControllerEvent {
override def state: ControllerState = ControllerState.PartitionReassignment
override def state: ControllerState = ControllerState.AlterPartitionReassignment
}
case object IsrChangeNotification extends ControllerEvent {
@ -1955,6 +2278,15 @@ case class ReplicaLeaderElection(
override def state: ControllerState = ControllerState.ManualLeaderBalance
}
/**
* @param partitionsOpt - an Optional set of partitions. If not present, all reassigning partitions are to be listed
*/
case class ListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]],
callback: ListReassignmentsCallback) extends ControllerEvent {
override def state: ControllerState = ControllerState.ListPartitionReassignment
}
// Used only in test cases
abstract class MockEvent(val state: ControllerState) extends ControllerEvent {
def process(): Unit

View File

@ -308,7 +308,7 @@ class ZkPartitionStateMachine(config: KafkaConfig,
if (code == Code.OK) {
controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(leaderIsrAndControllerEpoch.leaderAndIsr.isr,
partition, leaderIsrAndControllerEpoch, controllerContext.partitionReplicaAssignment(partition), isNew = true)
partition, leaderIsrAndControllerEpoch, controllerContext.partitionFullReplicaAssignment(partition), isNew = true)
successfulInitializations += partition
} else {
logFailedStateChange(partition, NewPartition, OnlinePartition, code)
@ -342,6 +342,9 @@ class ZkPartitionStateMachine(config: KafkaConfig,
}
finishedElections ++= finished
if (remaining.nonEmpty)
logger.info(s"Retrying leader election with strategy $partitionLeaderElectionStrategy for partitions $remaining")
}
finishedElections.toMap
@ -429,11 +432,11 @@ class ZkPartitionStateMachine(config: KafkaConfig,
adjustedLeaderAndIsrs, controllerContext.epoch, controllerContext.epochZkVersion)
finishedUpdates.foreach { case (partition, result) =>
result.right.foreach { leaderAndIsr =>
val replicas = controllerContext.partitionReplicaAssignment(partition)
val replicaAssignment = controllerContext.partitionFullReplicaAssignment(partition)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch)
controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(recipientsPerPartition(partition), partition,
leaderIsrAndControllerEpoch, replicas, isNew = false)
leaderIsrAndControllerEpoch, replicaAssignment, isNew = false)
}
}

View File

@ -175,7 +175,7 @@ class ZkReplicaStateMachine(config: KafkaConfig,
controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(replicaId),
replica.topicPartition,
leaderIsrAndControllerEpoch,
controllerContext.partitionReplicaAssignment(replica.topicPartition),
controllerContext.partitionFullReplicaAssignment(replica.topicPartition),
isNew = true)
logSuccessfulTransition(replicaId, partition, currentState, NewReplica)
controllerContext.putReplicaState(replica, NewReplica)
@ -202,7 +202,7 @@ class ZkReplicaStateMachine(config: KafkaConfig,
controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(replicaId),
replica.topicPartition,
leaderIsrAndControllerEpoch,
controllerContext.partitionReplicaAssignment(partition), isNew = false)
controllerContext.partitionFullReplicaAssignment(partition), isNew = false)
case None =>
}
}
@ -223,7 +223,7 @@ class ZkReplicaStateMachine(config: KafkaConfig,
controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(recipients,
partition,
leaderIsrAndControllerEpoch,
controllerContext.partitionReplicaAssignment(partition), isNew = false)
controllerContext.partitionFullReplicaAssignment(partition), isNew = false)
}
val replica = PartitionAndReplica(partition, replicaId)
val currentState = controllerContext.replicaState(replica)

View File

@ -245,8 +245,8 @@ class AdminManager(val config: KafkaConfig,
if (reassignPartitionsInProgress)
throw new ReassignmentInProgressException("A partition reassignment is in progress.")
val existingAssignment = zkClient.getReplicaAssignmentForTopics(immutable.Set(topic)).map {
case (topicPartition, replicas) => topicPartition.partition -> replicas
val existingAssignment = zkClient.getFullReplicaAssignmentForTopics(immutable.Set(topic)).map {
case (topicPartition, assignment) => topicPartition.partition -> assignment
}
if (existingAssignment.isEmpty)
throw new UnknownTopicOrPartitionException(s"The topic '$topic' does not exist.")
@ -261,7 +261,7 @@ class AdminManager(val config: KafkaConfig,
throw new InvalidPartitionsException(s"Topic already has $oldNumPartitions partitions.")
}
val reassignment = Option(newPartition.newAssignments).map(_.asScala.map(_.asScala.map(_.toInt))).map { assignments =>
val newPartitionsAssignment = Option(newPartition.newAssignments).map(_.asScala.map(_.asScala.map(_.toInt))).map { assignments =>
val unknownBrokers = assignments.flatten.toSet -- allBrokerIds
if (unknownBrokers.nonEmpty)
throw new InvalidReplicaAssignmentException(
@ -278,7 +278,7 @@ class AdminManager(val config: KafkaConfig,
}
val updatedReplicaAssignment = adminZkClient.addPartitions(topic, existingAssignment, allBrokers,
newPartition.totalCount, reassignment, validateOnly = validateOnly)
newPartition.totalCount, newPartitionsAssignment, validateOnly = validateOnly)
CreatePartitionsMetadata(topic, updatedReplicaAssignment, ApiError.NONE)
} catch {
case e: AdminOperationException =>

View File

@ -30,7 +30,7 @@ import kafka.api.ElectLeadersRequestOps
import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0}
import kafka.cluster.Partition
import kafka.common.OffsetAndMetadata
import kafka.controller.KafkaController
import kafka.controller.{KafkaController, PartitionReplicaAssignment}
import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult, LeaveGroupResult, SyncGroupResult}
import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator}
import kafka.message.ZStdCompressionCodec
@ -53,6 +53,7 @@ import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicR
import org.apache.kafka.common.message.DeleteGroupsResponseData
import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection}
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData
import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse}
import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData
import org.apache.kafka.common.message.DeleteTopicsResponseData
import org.apache.kafka.common.message.DeleteTopicsResponseData.{DeletableTopicResult, DeletableTopicResultCollection}
@ -2285,28 +2286,87 @@ class KafkaApis(val requestChannel: RequestChannel,
authorizeClusterOperation(request, ALTER)
val alterPartitionReassignmentsRequest = request.body[AlterPartitionReassignmentsRequest]
def sendResponseCallback(result: Either[Map[TopicPartition, ApiError], ApiError]): Unit = {
val responseData = result match {
case Right(topLevelError) =>
new AlterPartitionReassignmentsResponseData().setErrorMessage(topLevelError.message()).setErrorCode(topLevelError.error().code())
case Left(assignments) =>
val topicResponses = assignments.groupBy(_._1.topic()).map {
case (topic, reassignmentsByTp) =>
val partitionResponses = reassignmentsByTp.map {
case (topicPartition, error) =>
new ReassignablePartitionResponse().setPartitionIndex(topicPartition.partition())
.setErrorCode(error.error().code()).setErrorMessage(error.message())
}
new ReassignableTopicResponse().setName(topic).setPartitions(partitionResponses.toList.asJava)
}
new AlterPartitionReassignmentsResponseData().setResponses(topicResponses.toList.asJava)
}
sendResponseMaybeThrottle(request, requestThrottleMs =>
new AlterPartitionReassignmentsResponse(
new AlterPartitionReassignmentsResponseData().setThrottleTimeMs(requestThrottleMs)
.setErrorCode(Errors.UNSUPPORTED_VERSION.code()).setErrorMessage(Errors.UNSUPPORTED_VERSION.message())
.toStruct(0)
)
new AlterPartitionReassignmentsResponse(responseData.setThrottleTimeMs(requestThrottleMs))
)
}
val reassignments = alterPartitionReassignmentsRequest.data().topics().asScala.flatMap {
reassignableTopic => reassignableTopic.partitions().asScala.map {
reassignablePartition =>
val tp = new TopicPartition(reassignableTopic.name(), reassignablePartition.partitionIndex())
if (reassignablePartition.replicas() == null)
tp -> None // revert call
else
tp -> Some(reassignablePartition.replicas().asScala.map(_.toInt))
}
}.toMap
controller.alterPartitionReassignments(reassignments, sendResponseCallback)
}
def handleListPartitionReassignmentsRequest(request: RequestChannel.Request): Unit = {
authorizeClusterOperation(request, DESCRIBE)
val listPartitionReassignmentsRequest = request.body[ListPartitionReassignmentsRequest]
def sendResponseCallback(result: Either[Map[TopicPartition, PartitionReplicaAssignment], ApiError]): Unit = {
val responseData = result match {
case Right(error) => new ListPartitionReassignmentsResponseData().setErrorMessage(error.message()).setErrorCode(error.error().code())
case Left(assignments) =>
val topicReassignments = assignments.groupBy(_._1.topic()).map {
case (topic, reassignmentsByTp) =>
val partitionReassignments = reassignmentsByTp.map {
case (topicPartition, assignment) =>
new ListPartitionReassignmentsResponseData.OngoingPartitionReassignment()
.setPartitionIndex(topicPartition.partition())
.setAddingReplicas(assignment.addingReplicas.toList.asJava.asInstanceOf[java.util.List[java.lang.Integer]])
.setRemovingReplicas(assignment.removingReplicas.toList.asJava.asInstanceOf[java.util.List[java.lang.Integer]])
.setReplicas(assignment.replicas.toList.asJava.asInstanceOf[java.util.List[java.lang.Integer]])
}.toList
new ListPartitionReassignmentsResponseData.OngoingTopicReassignment().setName(topic)
.setPartitions(partitionReassignments.asJava)
}.toList
new ListPartitionReassignmentsResponseData().setTopics(topicReassignments.asJava)
}
sendResponseMaybeThrottle(request, requestThrottleMs =>
new ListPartitionReassignmentsResponse(
new ListPartitionReassignmentsResponseData().setThrottleTimeMs(requestThrottleMs)
.setErrorCode(Errors.UNSUPPORTED_VERSION.code()).setErrorMessage(Errors.UNSUPPORTED_VERSION.message())
.toStruct(0)
)
new ListPartitionReassignmentsResponse(responseData.setThrottleTimeMs(requestThrottleMs))
)
}
val partitionsOpt = listPartitionReassignmentsRequest.data().topics() match {
case topics: Any =>
Some(topics.iterator().asScala.flatMap { topic =>
topic.partitionIndexes().iterator().asScala
.map { tp => new TopicPartition(topic.name(), tp) }
}.toSet)
case _ => None
}
controller.listPartitionReassignments(partitionsOpt, sendResponseCallback)
}
private def configsAuthorizationApiError(resource: ConfigResource): ApiError = {
val error = resource.`type` match {
case ConfigResource.Type.BROKER | ConfigResource.Type.BROKER_LOGGER => Errors.CLUSTER_AUTHORIZATION_FAILED

View File

@ -20,6 +20,7 @@ import java.util.Properties
import kafka.admin.{AdminOperationException, AdminUtils, BrokerMetadata, RackAwareMode}
import kafka.common.TopicAlreadyMarkedForDeletionException
import kafka.controller.PartitionReplicaAssignment
import kafka.log.LogConfig
import kafka.server.{ConfigEntityName, ConfigType, DynamicConfig}
import kafka.utils._
@ -92,7 +93,8 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
zkClient.setOrCreateEntityConfigs(ConfigType.Topic, topic, config)
// create the partition assignment
writeTopicPartitionAssignment(topic, partitionReplicaAssignment, isUpdate = false)
writeTopicPartitionAssignment(topic, partitionReplicaAssignment.mapValues(PartitionReplicaAssignment(_, List(), List())).toMap,
isUpdate = false)
}
/**
@ -134,12 +136,12 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
LogConfig.validate(config)
}
private def writeTopicPartitionAssignment(topic: String, replicaAssignment: Map[Int, Seq[Int]], isUpdate: Boolean): Unit = {
private def writeTopicPartitionAssignment(topic: String, replicaAssignment: Map[Int, PartitionReplicaAssignment], isUpdate: Boolean): Unit = {
try {
val assignment = replicaAssignment.map { case (partitionId, replicas) => (new TopicPartition(topic,partitionId), replicas) }.toMap
if (!isUpdate) {
zkClient.createTopicAssignment(topic, assignment)
zkClient.createTopicAssignment(topic, assignment.mapValues(_.replicas).toMap)
} else {
zkClient.setTopicAssignment(topic, assignment)
}
@ -172,7 +174,7 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
* Add partitions to existing topic with optional replica assignment
*
* @param topic Topic for adding partitions to
* @param existingAssignment A map from partition id to its assigned replicas
* @param existingAssignment A map from partition id to its assignment
* @param allBrokers All brokers in the cluster
* @param numPartitions Number of partitions to be set
* @param replicaAssignment Manual replica assignment, or none
@ -180,7 +182,7 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
* @return the updated replica assignment
*/
def addPartitions(topic: String,
existingAssignment: Map[Int, Seq[Int]],
existingAssignment: Map[Int, PartitionReplicaAssignment],
allBrokers: Seq[BrokerMetadata],
numPartitions: Int = 1,
replicaAssignment: Option[Map[Int, Seq[Int]]] = None,
@ -188,7 +190,7 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
val existingAssignmentPartition0 = existingAssignment.getOrElse(0,
throw new AdminOperationException(
s"Unexpected existing replica assignment for topic '$topic', partition id 0 is missing. " +
s"Assignment: $existingAssignment"))
s"Assignment: $existingAssignment")).replicas
val partitionsToAdd = numPartitions - existingAssignment.size
if (partitionsToAdd <= 0)
@ -208,14 +210,16 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
startIndex, existingAssignment.size)
}
val proposedAssignment = existingAssignment ++ proposedAssignmentForNewPartitions
val proposedAssignment = existingAssignment ++ proposedAssignmentForNewPartitions.map { case (tp, replicas) =>
tp -> PartitionReplicaAssignment(replicas, List(), List())
}
if (!validateOnly) {
info(s"Creating $partitionsToAdd partitions for '$topic' with the following replica assignment: " +
s"$proposedAssignmentForNewPartitions.")
writeTopicPartitionAssignment(topic, proposedAssignment, isUpdate = true)
}
proposedAssignment
proposedAssignment.mapValues(_.replicas).toMap
}
private def validateReplicaAssignment(replicaAssignment: Map[Int, Seq[Int]],

View File

@ -21,7 +21,7 @@ import java.util.Properties
import com.yammer.metrics.core.MetricName
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker
import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch}
import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, PartitionReplicaAssignment}
import kafka.log.LogConfig
import kafka.metrics.KafkaMetricsGroup
import kafka.security.authorizer.AclAuthorizer.{NoAcls, VersionedAcls}
@ -484,7 +484,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @return SetDataResponse
*/
def setTopicAssignmentRaw(topic: String, assignment: collection.Map[TopicPartition, Seq[Int]], expectedControllerEpochZkVersion: Int): SetDataResponse = {
def setTopicAssignmentRaw(topic: String, assignment: collection.Map[TopicPartition, PartitionReplicaAssignment], expectedControllerEpochZkVersion: Int): SetDataResponse = {
val setDataRequest = SetDataRequest(TopicZNode.path(topic), TopicZNode.encode(assignment), ZkVersion.MatchAnyVersion)
retryRequestUntilConnected(setDataRequest, expectedControllerEpochZkVersion)
}
@ -496,7 +496,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @throws KeeperException if there is an error while setting assignment
*/
def setTopicAssignment(topic: String, assignment: Map[TopicPartition, Seq[Int]], expectedControllerEpochZkVersion: Int = ZkVersion.MatchAnyVersion) = {
def setTopicAssignment(topic: String, assignment: Map[TopicPartition, PartitionReplicaAssignment], expectedControllerEpochZkVersion: Int = ZkVersion.MatchAnyVersion) = {
val setDataResponse = setTopicAssignmentRaw(topic, assignment, expectedControllerEpochZkVersion)
setDataResponse.maybeThrow
}
@ -508,7 +508,8 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @throws KeeperException if there is an error while creating assignment
*/
def createTopicAssignment(topic: String, assignment: Map[TopicPartition, Seq[Int]]) = {
createRecursive(TopicZNode.path(topic), TopicZNode.encode(assignment))
val persistedAssignments = assignment.mapValues(PartitionReplicaAssignment(_, List(), List())).toMap
createRecursive(TopicZNode.path(topic), TopicZNode.encode(persistedAssignments))
}
/**
@ -569,18 +570,28 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
}
/**
* Gets the assignments for the given topics.
* Gets the replica assignments for the given topics.
* This function does not return information about which replicas are being added or removed from the assignment.
* @param topics the topics whose partitions we wish to get the assignments for.
* @return the replica assignment for each partition from the given topics.
*/
def getReplicaAssignmentForTopics(topics: Set[String]): Map[TopicPartition, Seq[Int]] = {
getFullReplicaAssignmentForTopics(topics).mapValues(_.replicas).toMap
}
/**
* Gets the replica assignments for the given topics.
* @param topics the topics whose partitions we wish to get the assignments for.
* @return the full replica assignment for each partition from the given topics.
*/
def getFullReplicaAssignmentForTopics(topics: Set[String]): Map[TopicPartition, PartitionReplicaAssignment] = {
val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic)))
val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq)
getDataResponses.flatMap { getDataResponse =>
val topic = getDataResponse.ctx.get.asInstanceOf[String]
getDataResponse.resultCode match {
case Code.OK => TopicZNode.decode(topic, getDataResponse.data)
case Code.NONODE => Map.empty[TopicPartition, Seq[Int]]
case Code.NONODE => Map.empty[TopicPartition, PartitionReplicaAssignment]
case _ => throw getDataResponse.resultException.get
}
}.toMap
@ -591,7 +602,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param topics the topics whose partitions we wish to get the assignments for.
* @return the partition assignment for each partition from the given topics.
*/
def getPartitionAssignmentForTopics(topics: Set[String]): Map[String, Map[Int, Seq[Int]]] = {
def getPartitionAssignmentForTopics(topics: Set[String]): Map[String, Map[Int, PartitionReplicaAssignment]] = {
val getDataRequests = topics.map(topic => GetDataRequest(TopicZNode.path(topic), ctx = Some(topic)))
val getDataResponses = retryRequestsUntilConnected(getDataRequests.toSeq)
getDataResponses.flatMap { getDataResponse =>
@ -600,7 +611,7 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
val partitionMap = TopicZNode.decode(topic, getDataResponse.data).map { case (k, v) => (k.partition, v) }
Map(topic -> partitionMap)
} else if (getDataResponse.resultCode == Code.NONODE) {
Map.empty[String, Map[Int, Seq[Int]]]
Map.empty[String, Map[Int, PartitionReplicaAssignment]]
} else {
throw getDataResponse.resultException.get
}
@ -791,7 +802,9 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
/**
* Returns all reassignments.
* @return the reassignments for each partition.
* @deprecated Use the PartitionReassignment Kafka API instead
*/
@Deprecated
def getPartitionReassignment: collection.Map[TopicPartition, Seq[Int]] = {
val getDataRequest = GetDataRequest(ReassignPartitionsZNode.path)
val getDataResponse = retryRequestUntilConnected(getDataRequest)
@ -815,7 +828,9 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* @param reassignment the reassignment to set on the reassignment znode
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @throws KeeperException if there is an error while setting or creating the znode
* @deprecated Use the PartitionReassignment Kafka API instead
*/
@Deprecated
def setOrCreatePartitionReassignment(reassignment: collection.Map[TopicPartition, Seq[Int]], expectedControllerEpochZkVersion: Int): Unit = {
def set(reassignmentData: Array[Byte]): SetDataResponse = {
@ -843,7 +858,9 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
* Creates the partition reassignment znode with the given reassignment.
* @param reassignment the reassignment to set on the reassignment znode.
* @throws KeeperException if there is an error while creating the znode
* @deprecated Use the PartitionReassignment Kafka API instead
*/
@Deprecated
def createPartitionReassignment(reassignment: Map[TopicPartition, Seq[Int]]) = {
createRecursive(ReassignPartitionsZNode.path, ReassignPartitionsZNode.encode(reassignment))
}
@ -851,7 +868,9 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
/**
* Deletes the partition reassignment znode.
* @param expectedControllerEpochZkVersion expected controller epoch zkVersion.
* @deprecated Use the PartitionReassignment Kafka API instead
*/
@Deprecated
def deletePartitionReassignment(expectedControllerEpochZkVersion: Int): Unit = {
deletePath(ReassignPartitionsZNode.path, expectedControllerEpochZkVersion)
}
@ -859,7 +878,9 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo
/**
* Checks if reassign partitions is in progress
* @return true if reassign partitions is in progress, else false
* @deprecated Use the PartitionReassignment Kafka API instead
*/
@Deprecated
def reassignPartitionsInProgress(): Boolean = {
pathExists(ReassignPartitionsZNode.path)
}

View File

@ -17,6 +17,7 @@
package kafka.zk
import java.nio.charset.StandardCharsets.UTF_8
import java.util
import java.util.Properties
import com.fasterxml.jackson.annotation.JsonProperty
@ -24,12 +25,13 @@ import com.fasterxml.jackson.core.JsonProcessingException
import kafka.api.{ApiVersion, KAFKA_0_10_0_IV1, LeaderAndIsr}
import kafka.cluster.{Broker, EndPoint}
import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener}
import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch}
import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, PartitionReplicaAssignment}
import kafka.security.auth.Resource.Separator
import kafka.security.authorizer.AclAuthorizer.VersionedAcls
import kafka.security.auth.{Acl, Resource, ResourceType}
import kafka.server.{ConfigType, DelegationTokenManager}
import kafka.utils.Json
import kafka.utils.json.JsonObject
import org.apache.kafka.common.{KafkaException, TopicPartition}
import org.apache.kafka.common.errors.UnsupportedVersionException
import org.apache.kafka.common.network.ListenerName
@ -43,7 +45,7 @@ import org.apache.zookeeper.data.{ACL, Stat}
import scala.beans.BeanProperty
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import scala.collection.{Map, Seq}
import scala.collection.{Map, Seq, mutable}
import scala.util.{Failure, Success, Try}
// This file contains objects for encoding/decoding data stored in ZooKeeper nodes (znodes).
@ -238,19 +240,49 @@ object TopicsZNode {
object TopicZNode {
def path(topic: String) = s"${TopicsZNode.path}/$topic"
def encode(assignment: collection.Map[TopicPartition, Seq[Int]]): Array[Byte] = {
val assignmentJson = assignment.map { case (partition, replicas) =>
partition.partition.toString -> replicas.asJava
def encode(assignment: collection.Map[TopicPartition, PartitionReplicaAssignment]): Array[Byte] = {
val replicaAssignmentJson = mutable.Map[String, util.List[Int]]()
val addingReplicasAssignmentJson = mutable.Map[String, util.List[Int]]()
val removingReplicasAssignmentJson = mutable.Map[String, util.List[Int]]()
for ((partition, replicaAssignment) <- assignment) {
replicaAssignmentJson += (partition.partition.toString -> replicaAssignment.replicas.asJava)
if (replicaAssignment.addingReplicas.nonEmpty)
addingReplicasAssignmentJson += (partition.partition.toString -> replicaAssignment.addingReplicas.asJava)
if (replicaAssignment.removingReplicas.nonEmpty)
removingReplicasAssignmentJson += (partition.partition.toString -> replicaAssignment.removingReplicas.asJava)
}
Json.encodeAsBytes(Map("version" -> 1, "partitions" -> assignmentJson.asJava).asJava)
Json.encodeAsBytes(Map(
"version" -> 2,
"partitions" -> replicaAssignmentJson.asJava,
"addingReplicas" -> addingReplicasAssignmentJson.asJava,
"removingReplicas" -> removingReplicasAssignmentJson.asJava
).asJava)
}
def decode(topic: String, bytes: Array[Byte]): Map[TopicPartition, Seq[Int]] = {
def decode(topic: String, bytes: Array[Byte]): Map[TopicPartition, PartitionReplicaAssignment] = {
def getReplicas(replicasJsonOpt: Option[JsonObject], partition: String): Seq[Int] = {
replicasJsonOpt match {
case Some(replicasJson) => replicasJson.get(partition) match {
case Some(ar) => ar.to[Seq[Int]]
case None => Seq.empty[Int]
}
case None => Seq.empty[Int]
}
}
Json.parseBytes(bytes).flatMap { js =>
val assignmentJson = js.asJsonObject
val partitionsJsonOpt = assignmentJson.get("partitions").map(_.asJsonObject)
val addingReplicasJsonOpt = assignmentJson.get("addingReplicas").map(_.asJsonObject)
val removingReplicasJsonOpt = assignmentJson.get("removingReplicas").map(_.asJsonObject)
partitionsJsonOpt.map { partitionsJson =>
partitionsJson.iterator.map { case (partition, replicas) =>
new TopicPartition(topic, partition.toInt) -> replicas.to[Seq[Int]]
new TopicPartition(topic, partition.toInt) -> PartitionReplicaAssignment(
replicas.to[Seq[Int]],
getReplicas(addingReplicasJsonOpt, partition),
getReplicas(addingReplicasJsonOpt, partition)
)
}
}
}.map(_.toMap).getOrElse(Map.empty)
@ -373,6 +405,10 @@ object DeleteTopicsTopicZNode {
def path(topic: String) = s"${DeleteTopicsZNode.path}/$topic"
}
/**
* The znode for initiating a partition reassignment.
* @deprecated Since 2.4, use the PartitionReassignment Kafka API instead.
*/
object ReassignPartitionsZNode {
/**
@ -388,14 +424,16 @@ object ReassignPartitionsZNode {
/**
* An assignment consists of a `version` and a list of `partitions`, which represent the
* assignment of topic-partitions to brokers.
* @deprecated Use the PartitionReassignment Kafka API instead
*/
case class PartitionAssignment(@BeanProperty @JsonProperty("version") version: Int,
@Deprecated
case class LegacyPartitionAssignment(@BeanProperty @JsonProperty("version") version: Int,
@BeanProperty @JsonProperty("partitions") partitions: java.util.List[ReplicaAssignment])
def path = s"${AdminZNode.path}/reassign_partitions"
def encode(reassignmentMap: collection.Map[TopicPartition, Seq[Int]]): Array[Byte] = {
val reassignment = PartitionAssignment(1,
val reassignment = LegacyPartitionAssignment(1,
reassignmentMap.toSeq.map { case (tp, replicas) =>
ReplicaAssignment(tp.topic, tp.partition, replicas.asJava)
}.asJava
@ -404,7 +442,7 @@ object ReassignPartitionsZNode {
}
def decode(bytes: Array[Byte]): Either[JsonProcessingException, collection.Map[TopicPartition, Seq[Int]]] =
Json.parseBytesAs[PartitionAssignment](bytes).right.map { partitionAssignment =>
Json.parseBytesAs[LegacyPartitionAssignment](bytes).right.map { partitionAssignment =>
partitionAssignment.partitions.asScala.iterator.map { replicaAssignment =>
new TopicPartition(replicaAssignment.topic, replicaAssignment.partition) -> replicaAssignment.replicas.asScala
}.toMap

View File

@ -1699,6 +1699,22 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
assertEquals(Option(broker3), TestUtils.currentLeader(client, partition2))
}
@Test
def testListReassignmentsDoesNotShowNonReassigningPartitions(): Unit = {
client = AdminClient.create(createConfig())
// Create topics
val topic = "list-reassignments-no-reassignments"
createTopic(topic, numPartitions = 1, replicationFactor = 3)
val tp = new TopicPartition(topic, 0)
val reassignmentsMap = client.listPartitionReassignments(Set(tp).asJava).reassignments().get()
assertEquals(0, reassignmentsMap.size())
val allReassignmentsMap = client.listPartitionReassignments().reassignments().get()
assertEquals(0, allReassignmentsMap.size())
}
@Test
def testValidIncrementalAlterConfigs(): Unit = {
client = AdminClient.create(createConfig)
@ -1877,6 +1893,42 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging {
Some("Invalid config value for resource"))
}
@Test
def testInvalidAlterPartitionReassignments(): Unit = {
client = AdminClient.create(createConfig)
val topic = "alter-reassignments-topic-1"
val tp1 = new TopicPartition(topic, 0)
val tp2 = new TopicPartition(topic, 1)
val tp3 = new TopicPartition(topic, 2)
createTopic(topic, numPartitions = 3)
val validAssignment = new NewPartitionReassignment((0 until brokerCount).map(_.asInstanceOf[Integer]).asJava)
val nonExistentTp1 = new TopicPartition("topicA", 0)
val nonExistentTp2 = new TopicPartition(topic, 3)
val nonExistentPartitionsResult = client.alterPartitionReassignments(Map(
tp1 -> java.util.Optional.of(validAssignment),
tp2 -> java.util.Optional.of(validAssignment),
tp3 -> java.util.Optional.of(validAssignment),
nonExistentTp1 -> java.util.Optional.of(validAssignment),
nonExistentTp2 -> java.util.Optional.of(validAssignment)
).asJava).values()
assertFutureExceptionTypeEquals(nonExistentPartitionsResult.get(nonExistentTp1), classOf[UnknownTopicOrPartitionException])
assertFutureExceptionTypeEquals(nonExistentPartitionsResult.get(nonExistentTp2), classOf[UnknownTopicOrPartitionException])
val extraNonExistentReplica = new NewPartitionReassignment((0 until brokerCount + 1).map(_.asInstanceOf[Integer]).asJava)
val negativeIdReplica = new NewPartitionReassignment(Seq(-3, -2, -1).map(_.asInstanceOf[Integer]).asJava)
val duplicateReplica = new NewPartitionReassignment(Seq(0, 1, 1).map(_.asInstanceOf[Integer]).asJava)
val invalidReplicaResult = client.alterPartitionReassignments(Map(
tp1 -> java.util.Optional.of(extraNonExistentReplica),
tp2 -> java.util.Optional.of(negativeIdReplica),
tp3 -> java.util.Optional.of(duplicateReplica)
).asJava).values()
assertFutureExceptionTypeEquals(invalidReplicaResult.get(tp1), classOf[InvalidReplicaAssignmentException])
assertFutureExceptionTypeEquals(invalidReplicaResult.get(tp2), classOf[InvalidReplicaAssignmentException])
assertFutureExceptionTypeEquals(invalidReplicaResult.get(tp3), classOf[InvalidReplicaAssignmentException])
}
@Test
def testLongTopicNames(): Unit = {
val client = AdminClient.create(createConfig)

View File

@ -381,8 +381,8 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
}
}
val existingAssignment = zkClient.getReplicaAssignmentForTopics(Set(topic)).map {
case (topicPartition, replicas) => topicPartition.partition -> replicas
val existingAssignment = zkClient.getFullReplicaAssignmentForTopics(Set(topic)).map {
case (topicPartition, assignment) => topicPartition.partition -> assignment
}
adminZkClient.addPartitions(topic, existingAssignment, adminZkClient.getBrokerMetadatas(), 2)
// read metadata from a broker and verify the new topic partitions exist

View File

@ -173,9 +173,9 @@ object ReplicationQuotasTestRig {
//Long stats
println("The replicas are " + replicas.toSeq.sortBy(_._1).map("\n" + _))
println("This is the current replica assignment:\n" + actual.toSeq)
println("This is the current replica assignment:\n" + actual.mapValues(_.replicas).toMap.toSeq)
println("proposed assignment is: \n" + newAssignment)
println("This is the assignment we ended up with" + actual)
println("This is the assignment we ended up with" + actual.mapValues(_.replicas).toMap)
//Test Stats
println(s"numBrokers: ${config.brokers}")

View File

@ -17,6 +17,7 @@
package kafka.admin
import kafka.controller.PartitionReplicaAssignment
import kafka.network.SocketServer
import org.junit.Assert._
import kafka.utils.TestUtils._
@ -38,24 +39,24 @@ class AddPartitionsTest extends BaseRequestTest {
val partitionId = 0
val topic1 = "new-topic1"
val topic1Assignment = Map(0->Seq(0,1))
val topic1Assignment = Map(0 -> PartitionReplicaAssignment(Seq(0,1), List(), List()))
val topic2 = "new-topic2"
val topic2Assignment = Map(0->Seq(1,2))
val topic2Assignment = Map(0 -> PartitionReplicaAssignment(Seq(1,2), List(), List()))
val topic3 = "new-topic3"
val topic3Assignment = Map(0->Seq(2,3,0,1))
val topic3Assignment = Map(0 -> PartitionReplicaAssignment(Seq(2,3,0,1), List(), List()))
val topic4 = "new-topic4"
val topic4Assignment = Map(0->Seq(0,3))
val topic4Assignment = Map(0 -> PartitionReplicaAssignment(Seq(0,3), List(), List()))
val topic5 = "new-topic5"
val topic5Assignment = Map(1->Seq(0,1))
val topic5Assignment = Map(1 -> PartitionReplicaAssignment(Seq(0,1), List(), List()))
@Before
override def setUp(): Unit = {
super.setUp()
createTopic(topic1, partitionReplicaAssignment = topic1Assignment)
createTopic(topic2, partitionReplicaAssignment = topic2Assignment)
createTopic(topic3, partitionReplicaAssignment = topic3Assignment)
createTopic(topic4, partitionReplicaAssignment = topic4Assignment)
createTopic(topic1, partitionReplicaAssignment = topic1Assignment.mapValues(_.replicas).toMap)
createTopic(topic2, partitionReplicaAssignment = topic2Assignment.mapValues(_.replicas).toMap)
createTopic(topic3, partitionReplicaAssignment = topic3Assignment.mapValues(_.replicas).toMap)
createTopic(topic4, partitionReplicaAssignment = topic4Assignment.mapValues(_.replicas).toMap)
}
@Test

View File

@ -29,7 +29,7 @@ import org.junit.{After, Test}
import kafka.admin.TopicCommand.ZookeeperTopicService
import kafka.common.TopicAlreadyMarkedForDeletionException
import kafka.controller.{OfflineReplica, PartitionAndReplica, ReplicaDeletionSuccessful}
import kafka.controller.{OfflineReplica, PartitionAndReplica, PartitionReplicaAssignment, ReplicaDeletionSuccessful}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException
import org.scalatest.Assertions.fail
@ -39,6 +39,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
var servers: Seq[KafkaServer] = Seq()
val expectedReplicaAssignment = Map(0 -> List(0, 1, 2))
val expectedReplicaFullAssignment = expectedReplicaAssignment.mapValues(PartitionReplicaAssignment(_, List(), List())).toMap
@After
override def tearDown(): Unit = {
@ -107,7 +108,6 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
@Test
def testPartitionReassignmentDuringDeleteTopic(): Unit = {
val expectedReplicaAssignment = Map(0 -> List(0, 1, 2))
val topic = "test"
val topicPartition = new TopicPartition(topic, 0)
val brokerConfigs = TestUtils.createBrokerConfigs(4, zkConnect, false)
@ -178,7 +178,6 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
@Test
def testIncreasePartitionCountDuringDeleteTopic(): Unit = {
val expectedReplicaAssignment = Map(0 -> List(0, 1, 2))
val topic = "test"
val topicPartition = new TopicPartition(topic, 0)
val brokerConfigs = TestUtils.createBrokerConfigs(4, zkConnect, false)
@ -248,7 +247,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
TestUtils.waitUntilTrue(() => zkClient.getBroker(follower.config.brokerId).isEmpty,
s"Follower ${follower.config.brokerId} was not removed from ZK")
// add partitions to topic
adminZkClient.addPartitions(topic, expectedReplicaAssignment, brokers, 2,
adminZkClient.addPartitions(topic, expectedReplicaFullAssignment, brokers, 2,
Some(Map(1 -> Seq(0, 1, 2), 2 -> Seq(0, 1, 2))))
// start topic deletion
adminZkClient.deleteTopic(topic)
@ -271,7 +270,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
adminZkClient.deleteTopic(topic)
// add partitions to topic
val newPartition = new TopicPartition(topic, 1)
adminZkClient.addPartitions(topic, expectedReplicaAssignment, brokers, 2,
adminZkClient.addPartitions(topic, expectedReplicaFullAssignment, brokers, 2,
Some(Map(1 -> Seq(0, 1, 2), 2 -> Seq(0, 1, 2))))
TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
// verify that new partition doesn't exist on any broker either

View File

@ -12,33 +12,37 @@
*/
package kafka.admin
import java.util.Collections
import java.util.Properties
import java.util.{Collections, Properties}
import kafka.admin.ReassignPartitionsCommand._
import kafka.common.AdminCommandFailedException
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.server.{DynamicConfig, KafkaConfig, KafkaServer}
import kafka.utils.TestUtils._
import kafka.utils.{Logging, TestUtils}
import kafka.zk.{ReassignPartitionsZNode, ZkVersion, ZooKeeperTestHarness}
import org.junit.Assert.{assertEquals, assertTrue}
import org.junit.Assert.{assertEquals, assertFalse, assertTrue}
import org.junit.{After, Before, Test}
import kafka.admin.ReplicationQuotaUtils._
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig}
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, ConfigEntry, NewPartitionReassignment, PartitionReassignment, AdminClient => JAdminClient}
import org.apache.kafka.common.{TopicPartition, TopicPartitionReplica}
import scala.collection.JavaConverters._
import scala.collection.Map
import scala.collection.Seq
import scala.collection.{Map, Seq}
import scala.util.Random
import java.io.File
import kafka.controller.PartitionReplicaAssignment
import kafka.log.LogConfig
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.errors.NoReassignmentInProgressException
class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
val partitionId = 0
var servers: Seq[KafkaServer] = null
var brokerIds: Seq[Int] = null
val topicName = "my-topic"
val tp0 = new TopicPartition(topicName, 0)
val tp1 = new TopicPartition(topicName, 1)
val delayMs = 1000
var adminClient: Admin = null
@ -49,8 +53,9 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
super.setUp()
}
def startBrokers(brokerIds: Seq[Int]): Unit = {
servers = brokerIds.map { i =>
def startBrokers(ids: Seq[Int]): Unit = {
brokerIds = ids
servers = ids.map { i =>
val props = createBrokerConfig(i, zkConnect, enableControlledShutdown = false, logDirCount = 3)
// shorter backoff to reduce test durations when no active partitions are eligible for fetching due to throttling
props.put(KafkaConfig.ReplicaFetchBackoffMsProp, "100")
@ -62,7 +67,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
val props = new Properties()
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers))
props.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "10000")
AdminClient.create(props)
JAdminClient.create(props)
}
def getRandomLogDirAssignment(brokerId: Int): String = {
@ -86,26 +91,27 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
//Given a single replica on server 100
startBrokers(Seq(100, 101, 102))
adminClient = createAdminClient(servers)
createTopic(zkClient, topicName, Map(0 -> Seq(100)), servers = servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
val topicPartition = new TopicPartition(topicName, 0)
val leaderServer = servers.find(_.config.brokerId == 100).get
leaderServer.replicaManager.logManager.truncateFullyAndStartAt(topicPartition, 100L, false)
leaderServer.replicaManager.logManager.truncateFullyAndStartAt(tp0, 100L, false)
val topicJson: String = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101, 102]}]}"""
val topicJson = executeAssignmentJson(Seq(
PartitionAssignmentJson(tp0, replicas=Seq(101, 102))
))
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
val newLeaderServer = servers.find(_.config.brokerId == 101).get
TestUtils.waitUntilTrue (
() => newLeaderServer.replicaManager.nonOfflinePartition(topicPartition).flatMap(_.leaderLogIfLocal).isDefined,
waitUntilTrue (
() => newLeaderServer.replicaManager.nonOfflinePartition(tp0).flatMap(_.leaderLogIfLocal).isDefined,
"broker 101 should be the new leader", pause = 1L
)
assertEquals(100, newLeaderServer.replicaManager.localLogOrException(topicPartition)
assertEquals(100, newLeaderServer.replicaManager.localLogOrException(tp0)
.highWatermark)
val newFollowerServer = servers.find(_.config.brokerId == 102).get
TestUtils.waitUntilTrue(() => newFollowerServer.replicaManager.localLogOrException(topicPartition)
waitUntilTrue(() => newFollowerServer.replicaManager.localLogOrException(tp0)
.highWatermark == 100,
"partition follower's highWatermark should be 100")
}
@ -115,18 +121,20 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
//Given a single replica on server 100
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
val partition = 0
// Get a random log directory on broker 101
val expectedLogDir = getRandomLogDirAssignment(101)
createTopic(zkClient, topicName, Map(partition -> Seq(100)), servers = servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
//When we move the replica on 100 to broker 101
val topicJson: String = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101],"log_dirs":["$expectedLogDir"]}]}"""
val topicJson = executeAssignmentJson(Seq(
PartitionAssignmentJson(tp0, replicas = Seq(101), logDirectories = Some(Seq(expectedLogDir)))
))
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
waitForReassignmentToComplete()
waitForZkReassignmentToComplete()
//Then the replica should be on 101
assertEquals(Seq(101), zkClient.getPartitionAssignmentForTopics(Set(topicName)).get(topicName).get(partition))
val partitionAssignment = zkClient.getPartitionAssignmentForTopics(Set(topicName)).get(topicName).get(tp0.partition())
assertMoveForPartitionOccurred(Seq(101), partitionAssignment)
// The replica should be in the expected log directory on broker 101
val replica = new TopicPartitionReplica(topicName, 0, 101)
assertEquals(expectedLogDir, adminClient.describeReplicaLogDirs(Collections.singleton(replica)).all().get.get(replica).getCurrentReplicaLogDir)
@ -138,13 +146,15 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
val expectedLogDir = getRandomLogDirAssignment(100)
createTopic(zkClient, topicName, Map(0 -> Seq(100)), servers = servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
// When we execute an assignment that moves an existing replica to another log directory on the same broker
val topicJson: String = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[100],"log_dirs":["$expectedLogDir"]}]}"""
val topicJson = executeAssignmentJson(Seq(
PartitionAssignmentJson(tp0, replicas = Seq(100), logDirectories = Some(Seq(expectedLogDir)))
))
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
val replica = new TopicPartitionReplica(topicName, 0, 100)
TestUtils.waitUntilTrue(() => {
waitUntilTrue(() => {
expectedLogDir == adminClient.describeReplicaLogDirs(Collections.singleton(replica)).all().get.get(replica).getCurrentReplicaLogDir
}, "Partition should have been moved to the expected log directory", 1000)
}
@ -161,7 +171,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
), servers = servers)
//When rebalancing
val newAssignment = generateAssignment(zkClient, brokers, json(topicName), true)._1
val newAssignment = generateAssignment(zkClient, brokers, generateAssignmentJson(topicName), true)._1
// Find a partition in the new assignment on broker 102 and a random log directory on broker 102,
// which currently does not have any partition for this topic
val partition1 = newAssignment.find { case (_, brokerIds) => brokerIds.contains(102) }.get._1.partition
@ -177,11 +187,11 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
val newReplicaAssignment = Map(replica1 -> expectedLogDir1, replica2 -> expectedLogDir2)
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient),
ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, newReplicaAssignment), NoThrottle)
waitForReassignmentToComplete()
waitForZkReassignmentToComplete()
// Then the replicas should span all three brokers
val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
assertEquals(Seq(100, 101, 102), actual.values.flatten.toSeq.distinct.sorted)
assertMoveForTopicOccurred(Seq(100, 101, 102), actual)
// The replica should be in the expected log directory on broker 102 and 100
waitUntilTrue(() => {
expectedLogDir1 == adminClient.describeReplicaLogDirs(Collections.singleton(replica1)).all().get.get(replica1).getCurrentReplicaLogDir
@ -203,14 +213,14 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
), servers = servers)
//When rebalancing
val newAssignment = generateAssignment(zkClient, Array(100, 101), json(topicName), true)._1
val newAssignment = generateAssignment(zkClient, Array(100, 101), generateAssignmentJson(topicName), true)._1
ReassignPartitionsCommand.executeAssignment(zkClient, None,
ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty), NoThrottle)
waitForReassignmentToComplete()
waitForZkReassignmentToComplete()
//Then replicas should only span the first two brokers
val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
assertEquals(Seq(100, 101), actual.values.flatten.toSeq.distinct.sorted)
assertMoveForTopicOccurred(Seq(100, 101), actual)
}
@Test
@ -247,16 +257,16 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
//When rebalancing
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient),
ReassignPartitionsCommand.formatAsReassignmentJson(proposed, proposedReplicaAssignment), NoThrottle)
waitForReassignmentToComplete()
waitForZkReassignmentToComplete()
//Then the proposed changes should have been made
val actual = zkClient.getPartitionAssignmentForTopics(Set("topic1", "topic2"))
assertEquals(Seq(100, 102), actual("topic1")(0))//changed
assertEquals(Seq(101, 102), actual("topic1")(1))
assertEquals(Seq(100, 102), actual("topic1")(2))//changed
assertEquals(Seq(100, 101), actual("topic2")(0))
assertEquals(Seq(101, 100), actual("topic2")(1))//changed
assertEquals(Seq(100, 102), actual("topic2")(2))//changed
assertMoveForPartitionOccurred(Seq(100, 102), actual("topic1")(0)) //changed
assertMoveForPartitionOccurred(Seq(101, 102), actual("topic1")(1))
assertMoveForPartitionOccurred(Seq(100, 102), actual("topic1")(2)) //changed
assertMoveForPartitionOccurred(Seq(100, 101), actual("topic2")(0))
assertMoveForPartitionOccurred(Seq(101, 100), actual("topic2")(1)) //changed
assertMoveForPartitionOccurred(Seq(100, 102), actual("topic2")(2)) //changed
// The replicas should be in the expected log directories
val replicaDirs = adminClient.describeReplicaLogDirs(List(replica1, replica2).asJava).all().get()
@ -283,7 +293,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
assertEquals(expectedDurationSecs, numMessages * msgSize / initialThrottle.interBrokerLimit)
//Start rebalance which will move replica on 100 -> replica on 102
val newAssignment = generateAssignment(zkClient, Array(101, 102), json(topicName), true)._1
val newAssignment = generateAssignment(zkClient, Array(101, 102), generateAssignmentJson(topicName), true)._1
val start = System.currentTimeMillis()
ReassignPartitionsCommand.executeAssignment(zkClient, None,
@ -293,12 +303,12 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
checkThrottleConfigAddedToZK(adminZkClient, initialThrottle.interBrokerLimit, servers, topicName, Set("0:100","0:101"), Set("0:102"))
//Await completion
waitForReassignmentToComplete()
waitForZkReassignmentToComplete()
val took = System.currentTimeMillis() - start - delayMs
//Check move occurred
val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
assertEquals(Seq(101, 102), actual.values.flatten.toSeq.distinct.sorted)
assertMoveForTopicOccurred(Seq(101, 102), actual)
//Then command should have taken longer than the throttle rate
assertTrue(s"Expected replication to be > ${expectedDurationSecs * 0.9 * 1000} but was $took",
@ -367,7 +377,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
produceMessages(topicName, numMessages = 200, acks = 0, valueLength = 100 * 1000)
//Start rebalance
val newAssignment = generateAssignment(zkClient, Array(101, 102), json(topicName), true)._1
val newAssignment = generateAssignment(zkClient, Array(101, 102), generateAssignmentJson(topicName), true)._1
ReassignPartitionsCommand.executeAssignment(zkClient, None,
ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty), Throttle(initialThrottle))
@ -391,7 +401,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
checkThrottleConfigAddedToZK(adminZkClient, newThrottle, servers, topicName, Set("0:100","0:101"), Set("0:102"))
//Await completion
waitForReassignmentToComplete()
waitForZkReassignmentToComplete()
//Verify should remove the throttle
verifyAssignment(zkClient, None, ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty))
@ -401,17 +411,17 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
//Check move occurred
val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
assertEquals(Seq(101, 102), actual.values.flatten.toSeq.distinct.sorted)
assertMoveForTopicOccurred(Seq(101, 102), actual)
}
@Test(expected = classOf[AdminCommandFailedException])
def shouldFailIfProposedDoesNotMatchExisting(): Unit = {
//Given a single replica on server 100
startBrokers(Seq(100, 101))
createTopic(zkClient, topicName, Map(0 -> Seq(100)), servers = servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
//When we execute an assignment that includes an invalid partition (1:101 in this case)
val topicJson = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":1,"replicas":[101]}]}"""
val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp1, Seq(101))))
ReassignPartitionsCommand.executeAssignment(zkClient, None, topicJson, NoThrottle)
}
@ -419,10 +429,10 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
def shouldFailIfProposedHasEmptyReplicaList(): Unit = {
//Given a single replica on server 100
startBrokers(Seq(100, 101))
createTopic(zkClient, topicName, Map(0 -> Seq(100)), servers = servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
//When we execute an assignment that specifies an empty replica list (0: empty list in this case)
val topicJson = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[]}]}"""
val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp0, Seq())))
ReassignPartitionsCommand.executeAssignment(zkClient, None, topicJson, NoThrottle)
}
@ -430,10 +440,10 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
def shouldFailIfProposedHasInvalidBrokerID(): Unit = {
//Given a single replica on server 100
startBrokers(Seq(100, 101))
createTopic(zkClient, topicName, Map(0 -> Seq(100)), servers = servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
//When we execute an assignment that specifies an invalid brokerID (102: invalid broker ID in this case)
val topicJson = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101, 102]}]}"""
val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp0, Seq(101, 102))))
ReassignPartitionsCommand.executeAssignment(zkClient, None, topicJson, NoThrottle)
}
@ -442,10 +452,10 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
// Given a single replica on server 100
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
createTopic(zkClient, topicName, Map(0 -> Seq(100)), servers = servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
// When we execute an assignment that specifies an invalid log directory
val topicJson: String = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101],"log_dirs":["invalidDir"]}]}"""
val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp0, Seq(101), logDirectories = Some(Seq("invalidDir")))))
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
}
@ -455,10 +465,10 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
val logDir = getRandomLogDirAssignment(100)
createTopic(zkClient, topicName, Map(0 -> Seq(100)), servers = servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
// When we execute an assignment whose length of replicas doesn't match that of replicas
val topicJson: String = s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101],"log_dirs":["$logDir", "$logDir"]}]}"""
// When we execute an assignment whose length of replicas doesn't match that of log dirs
val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp0, Seq(101), logDirectories = Some(Seq(logDir, logDir)))))
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
}
@ -485,7 +495,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
//When we run a throttled reassignment
new ReassignPartitionsCommand(zkClient, None, move, adminZkClient = adminZkClient).reassignPartitions(throttle)
waitForReassignmentToComplete()
waitForZkReassignmentToComplete()
//Check moved replicas did move
assertEquals(Seq(0, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
@ -525,7 +535,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
new ReassignPartitionsCommand(zkClient, None, firstMove, adminZkClient = adminZkClient).reassignPartitions()
// Low pause to detect deletion of the reassign_partitions znode before the reassignment is complete
waitForReassignmentToComplete(pause = 1L)
waitForZkReassignmentToComplete(pause = 1L)
// Check moved replicas did move
assertEquals(Seq(0, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
@ -550,7 +560,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
new ReassignPartitionsCommand(zkClient, None, secondMove, adminZkClient = adminZkClient).reassignPartitions()
// Low pause to detect deletion of the reassign_partitions znode before the reassignment is complete
waitForReassignmentToComplete(pause = 1L)
waitForZkReassignmentToComplete(pause = 1L)
// Check moved replicas did move
assertEquals(Seq(0, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
@ -584,7 +594,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
}.exists(identity)
// Low pause to detect deletion of the reassign_partitions znode before the reassignment is complete
waitForReassignmentToComplete(pause = 1L)
waitForZkReassignmentToComplete(pause = 1L)
// Check moved replicas for thirdMove and fourthMove
assertEquals(Seq(1, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
@ -620,23 +630,611 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
zkClient.setOrCreatePartitionReassignment(firstMove, ZkVersion.MatchAnyVersion)
servers.foreach(_.startup())
waitForReassignmentToComplete()
waitForZkReassignmentToComplete()
assertEquals(Seq(2, 1), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1)))
assertEquals(Seq.empty, zkClient.getReplicasForPartition(new TopicPartition("customers", 0)))
}
def waitForReassignmentToComplete(pause: Long = 100L): Unit = {
/**
* Set a reassignment through the `/topics/<topic>` znode and set the `reassign_partitions` znode while the brokers are down.
* Verify that the reassignment is triggered by the Controller during start-up with the `reassign_partitions` znode taking precedence
*/
@Test
def shouldTriggerReassignmentWithZnodePrecedenceOnControllerStartup(): Unit = {
startBrokers(Seq(0, 1, 2))
adminClient = createAdminClient(servers)
createTopic(zkClient, "orders", Map(0 -> List(0, 1), 1 -> List(1, 2), 2 -> List(0, 1), 3 -> List(0, 1)), servers)
val sameMoveTp = new TopicPartition("orders", 2)
// Throttle to ensure we minimize race conditions and test flakiness
throttle(Seq("orders"), throttleSettingForSeconds(10), Map(
sameMoveTp -> Seq(0, 1, 2)
))
servers.foreach(_.shutdown())
adminClient.close()
zkClient.setTopicAssignment("orders", Map(
new TopicPartition("orders", 0) -> PartitionReplicaAssignment(List(0, 1), List(2), List(0)), // should be overwritten
new TopicPartition("orders", 1) -> PartitionReplicaAssignment(List(1, 2), List(3), List(1)), // should be overwritten
// should be overwritten (so we know to remove it from ZK) even though we do the exact same move
sameMoveTp -> PartitionReplicaAssignment(List(0, 1, 2), List(2), List(0)),
new TopicPartition("orders", 3) -> PartitionReplicaAssignment(List(0, 1, 2), List(2), List(0)) // moves
))
val move = Map(
new TopicPartition("orders", 0) -> Seq(2, 1), // moves
new TopicPartition("orders", 1) -> Seq(1, 2), // stays
sameMoveTp -> Seq(1, 2), // same reassignment
// orders-3 intentionally left for API
new TopicPartition("customers", 0) -> Seq(1, 2) // non-existent topic, triggers topic deleted path
)
// Set znode directly to avoid non-existent topic validation
zkClient.setOrCreatePartitionReassignment(move, ZkVersion.MatchAnyVersion)
servers.foreach(_.startup())
TestUtils.waitUntilBrokerMetadataIsPropagated(servers)
adminClient = createAdminClient(servers)
resetBrokersThrottle()
waitForZkReassignmentToComplete()
assertEquals(Seq(2, 1), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1)))
assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(sameMoveTp))
assertEquals(Seq.empty, zkClient.getReplicasForPartition(new TopicPartition("customers", 0)))
}
@Test
def shouldListReassignmentsTriggeredByZk(): Unit = {
// Given a single replica on server 100
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
// Get a random log directory on broker 101
val expectedLogDir = getRandomLogDirAssignment(101)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
// Given throttle set so replication will take at least 2 sec (to ensure we don't minimize race condition and test flakiness
val throttle: Long = 1000 * 1000
produceMessages(topicName, numMessages = 20, acks = 0, valueLength = 100 * 1000)
// When we move the replica on 100 to broker 101
val topicJson = executeAssignmentJson(Seq(
PartitionAssignmentJson(tp0, replicas = Seq(101), Some(Seq(expectedLogDir)))))
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, Throttle(throttle))
// Then the replica should be removing
val reassigningPartitionsResult = adminClient.listPartitionReassignments(Set(tp0).asJava).reassignments().get().get(tp0)
assertIsReassigning(from = Seq(100), to = Seq(101), reassigningPartitionsResult)
waitForZkReassignmentToComplete()
// Then the replica should be on 101
val partitionAssignment = zkClient.getPartitionAssignmentForTopics(Set(topicName)).get(topicName).get(tp0.partition())
assertMoveForPartitionOccurred(Seq(101), partitionAssignment)
}
@Test
def shouldReassignThroughApi(): Unit = {
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
assertTrue(adminClient.listPartitionReassignments(Set(tp0).asJava).reassignments().get().isEmpty)
assertEquals(Seq(100), zkClient.getReplicasForPartition(tp0))
adminClient.alterPartitionReassignments(
Map(reassignmentEntry(tp0, Seq(101))).asJava
).all().get()
waitForAllReassignmentsToComplete()
assertEquals(Seq(101), zkClient.getReplicasForPartition(tp0))
}
@Test
def shouldListMovingPartitionsThroughApi(): Unit = {
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
val topic2 = "topic2"
val tp2 = new TopicPartition(topic2, 0)
createTopic(zkClient, topicName,
Map(tp0.partition() -> Seq(100),
tp1.partition() -> Seq(101)),
servers = servers)
createTopic(zkClient, topic2,
Map(tp2.partition() -> Seq(100)),
servers = servers)
assertTrue(adminClient.listPartitionReassignments().reassignments().get().isEmpty)
// Throttle to ensure we minimize race conditions and test flakiness
throttle(Seq(topicName), throttleSettingForSeconds(10), Map(
tp0 -> Seq(100, 101),
tp2 -> Seq(100, 101)
))
adminClient.alterPartitionReassignments(
Map(reassignmentEntry(tp0, Seq(101)),
reassignmentEntry(tp2, Seq(101))).asJava
).all().get()
val reassignmentsInProgress = adminClient.listPartitionReassignments(Set(tp0, tp1, tp2).asJava).reassignments().get()
assertFalse(reassignmentsInProgress.containsKey(tp1)) // tp1 is not reassigning
assertIsReassigning(from = Seq(100), to = Seq(101), reassignmentsInProgress.get(tp0))
assertIsReassigning(from = Seq(100), to = Seq(101), reassignmentsInProgress.get(tp2))
resetBrokersThrottle()
waitForAllReassignmentsToComplete()
assertEquals(Seq(101), zkClient.getReplicasForPartition(tp0))
assertEquals(Seq(101), zkClient.getReplicasForPartition(tp2))
}
@Test
def shouldUseLatestOrderingIfTwoConsecutiveReassignmentsHaveSameSetButDifferentOrdering(): Unit = {
startBrokers(Seq(100, 101, 102))
adminClient = createAdminClient(servers)
createTopic(zkClient, topicName,
Map(tp0.partition() -> Seq(100, 101),
tp1.partition() -> Seq(100, 101)),
servers = servers)
// Throttle to ensure we minimize race conditions and test flakiness
throttle(Seq(topicName), throttleSettingForSeconds(10), Map(
tp0 -> Seq(100, 101, 102),
tp1 -> Seq(100, 101, 102)
))
adminClient.alterPartitionReassignments(
Map(reassignmentEntry(tp0, Seq(100, 101, 102)),
reassignmentEntry(tp1, Seq(100, 101, 102))).asJava
).all().get()
val apiReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get()
assertIsReassigning(from = Seq(100, 101), to = Seq(100, 101, 102), apiReassignmentsInProgress.get(tp0))
assertIsReassigning(from = Seq(100, 101), to = Seq(100, 101, 102), apiReassignmentsInProgress.get(tp1))
// API reassignment to the same replicas but a different order
adminClient.alterPartitionReassignments(
Map(reassignmentEntry(tp0, Seq(102, 101, 100)),
reassignmentEntry(tp1, Seq(102, 101, 100))).asJava
).all().get()
val apiReassignmentsInProgress2 = adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get()
// assert same replicas, ignoring ordering
assertIsReassigning(from = Seq(100, 101), to = Seq(100, 101, 102), apiReassignmentsInProgress2.get(tp0))
assertIsReassigning(from = Seq(100, 101), to = Seq(100, 101, 102), apiReassignmentsInProgress2.get(tp1))
resetBrokersThrottle()
waitForAllReassignmentsToComplete()
//Check move occurred
val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
assertMoveForPartitionOccurred(Seq(102, 101, 100), actual(tp0.partition()))
assertMoveForPartitionOccurred(Seq(102, 101, 100), actual(tp1.partition()))
}
/**
* 1. Trigger API reassignment for partitions
* 2. Trigger ZK reassignment for partitions
* Ensure ZK reassignment overrides API reassignment and znode is deleted
*/
@Test
def znodeReassignmentShouldOverrideApiTriggeredReassignment(): Unit = {
startBrokers(Seq(100, 101, 102))
adminClient = createAdminClient(servers)
createTopic(zkClient, topicName,
Map(tp0.partition() -> Seq(100),
tp1.partition() -> Seq(100)),
servers = servers)
// Throttle to avoid race conditions
val throttleSetting = throttleSettingForSeconds(10)
throttle(Seq(topicName), throttleSetting, Map(
tp0 -> Seq(100, 101, 102),
tp1 -> Seq(100, 101, 102)
))
// API reassignment to 101 for both partitions
adminClient.alterPartitionReassignments(
Map(reassignmentEntry(tp0, Seq(101)),
reassignmentEntry(tp1, Seq(101))).asJava
).all().get()
val apiReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get()
assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress.get(tp0))
assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress.get(tp1))
// znode reassignment to 102 for both partitions
val topicJson = executeAssignmentJson(Seq(
PartitionAssignmentJson(tp0, Seq(102)),
PartitionAssignmentJson(tp1, Seq(102))
))
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, Throttle(throttleSetting.throttleBytes.toLong))
waitUntilTrue(() => {
!adminClient.listPartitionReassignments().reassignments().get().isEmpty
}, "Controller should have picked up on znode creation", 1000)
val zkReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get()
assertIsReassigning(from = Seq(100), to = Seq(102), zkReassignmentsInProgress.get(tp0))
assertIsReassigning(from = Seq(100), to = Seq(102), zkReassignmentsInProgress.get(tp1))
resetBrokersThrottle()
waitForZkReassignmentToComplete()
assertTrue(adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get().isEmpty)
assertEquals(Seq(102), zkClient.getReplicasForPartition(tp0))
assertEquals(Seq(102), zkClient.getReplicasForPartition(tp1))
}
/**
* 1. Trigger ZK reassignment for TP A-0, A-1
* 2. Trigger API reassignment for partitions TP A-1, B-0
* 3. Unthrottle A-0, A-1 so the ZK reassignment finishes quickly
* 4. Ensure ZK node is emptied out after the API reassignment of 1 finishes
*/
@Test
def shouldDeleteReassignmentZnodeAfterApiReassignmentForPartitionCompletes(): Unit = {
startBrokers(Seq(100, 101, 102))
adminClient = createAdminClient(servers)
val tpA0 = new TopicPartition("A", 0)
val tpA1 = new TopicPartition("A", 1)
val tpB0 = new TopicPartition("B", 0)
createTopic(zkClient, "A",
Map(tpA0.partition() -> Seq(100),
tpA1.partition() -> Seq(100)),
servers = servers)
createTopic(zkClient, "B",
Map(tpB0.partition() -> Seq(100)),
servers = servers)
// Throttle to avoid race conditions
throttle(Seq("A", "B"), throttleSettingForSeconds(10), Map(
tpA0 -> Seq(100, 101, 102),
tpA1 -> Seq(100, 101, 102),
tpB0 -> Seq(100, 101, 102)
))
// 1. znode reassignment to 101 for TP A-0, A-1
val topicJson = executeAssignmentJson(Seq(
PartitionAssignmentJson(tpA0, replicas=Seq(101)),
PartitionAssignmentJson(tpA1, replicas=Seq(101))
))
ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
waitUntilTrue(() => {
!adminClient.listPartitionReassignments().reassignments().get().isEmpty
}, "Controller should have picked up on znode creation", 1000)
val zkReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tpA0, tpA1).asJava).reassignments().get()
assertIsReassigning(from = Seq(100), to = Seq(101), zkReassignmentsInProgress.get(tpA0))
assertIsReassigning(from = Seq(100), to = Seq(101), zkReassignmentsInProgress.get(tpA1))
// 2. API reassignment to 102 for TP A-1, B-0
adminClient.alterPartitionReassignments(
Map(reassignmentEntry(tpA1, Seq(102)), reassignmentEntry(tpB0, Seq(102))).asJava
).all().get()
val apiReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tpA1, tpB0).asJava).reassignments().get()
assertIsReassigning(from = Seq(100), to = Seq(102), apiReassignmentsInProgress.get(tpA1))
assertIsReassigning(from = Seq(100), to = Seq(102), apiReassignmentsInProgress.get(tpB0))
// 3. Unthrottle topic A
removePartitionReplicaThrottles(Set(tpA0, tpA1))
waitForZkReassignmentToComplete()
// 4. Ensure the API reassignment not part of the znode is still in progress
val leftoverReassignments = adminClient.listPartitionReassignments(Set(tpA0, tpA1, tpB0).asJava).reassignments().get()
assertEquals(1, leftoverReassignments.size())
val tpB0LeftoverReassignment = leftoverReassignments.get(tpB0)
assertIsReassigning(from = Seq(100), to = Seq(102), tpB0LeftoverReassignment)
resetBrokersThrottle()
waitForAllReassignmentsToComplete()
assertEquals(Seq(101), zkClient.getReplicasForPartition(tpA0))
assertEquals(Seq(102), zkClient.getReplicasForPartition(tpA1))
assertEquals(Seq(102), zkClient.getReplicasForPartition(tpB0))
}
@Test
def shouldBeAbleToCancelThroughApi(): Unit = {
startBrokers(Seq(100, 101, 102))
adminClient = createAdminClient(servers)
createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100, 101)), servers = servers)
// Throttle to ensure we minimize race conditions and test flakiness
throttle(Seq(topicName), throttleSettingForSeconds(10), Map(
tp0 -> Seq(100, 101, 102)
))
// move to [102, 101]
adminClient.alterPartitionReassignments(
Map(reassignmentEntry(tp0, Seq(102, 101))).asJava
).all().get()
val apiReassignmentsInProgress = adminClient.listPartitionReassignments().reassignments().get()
val tpReassignment = apiReassignmentsInProgress.get(tp0)
assertIsReassigning(from = Seq(100, 101), to = Seq(101, 102), tpReassignment)
adminClient.alterPartitionReassignments(
Map(cancelReassignmentEntry(tp0)).asJava
).all().get()
resetBrokersThrottle()
waitForAllReassignmentsToComplete()
assertEquals(Seq(100, 101), zkClient.getReplicasForPartition(tp0).sorted) // revert ordering is not guaranteed
}
@Test
def shouldBeAbleToCancelZkTriggeredReassignmentThroughApi(): Unit = {
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
createTopic(zkClient, topicName,
Map(tp0.partition() -> Seq(100),
tp1.partition() -> Seq(100)),
servers = servers)
// Throttle to avoid race conditions
throttle(Seq(topicName), throttleSettingForSeconds(10), Map(
tp0 -> Seq(100, 101),
tp1 -> Seq(100, 101)
))
val move = Map(
tp0 -> Seq(101),
tp1 -> Seq(101)
)
zkClient.setOrCreatePartitionReassignment(move, ZkVersion.MatchAnyVersion)
waitUntilTrue(() => {
!adminClient.listPartitionReassignments().reassignments().get().isEmpty
}, "Controller should have picked up on znode creation", 1000)
var reassignmentIsOngoing = adminClient.listPartitionReassignments().reassignments().get().size() > 0
assertTrue(reassignmentIsOngoing)
adminClient.alterPartitionReassignments(
Map(cancelReassignmentEntry(tp0), cancelReassignmentEntry(tp1)).asJava
).all().get()
resetBrokersThrottle()
waitForZkReassignmentToComplete()
reassignmentIsOngoing = adminClient.listPartitionReassignments().reassignments().get().size() > 0
assertFalse(reassignmentIsOngoing)
assertEquals(Seq(100), zkClient.getReplicasForPartition(tp0))
assertEquals(Seq(100), zkClient.getReplicasForPartition(tp1))
}
/**
* Cancel and set reassignments in the same API call.
* Even though one cancellation is invalid, ensure the other entries in the request pass
*/
@Test
def testCancelAndSetSomeReassignments(): Unit = {
startBrokers(Seq(100, 101, 102))
adminClient = createAdminClient(servers)
val tp2 = new TopicPartition(topicName, 2)
val tp3 = new TopicPartition(topicName, 3)
createTopic(zkClient, topicName,
Map(tp0.partition() -> Seq(100), tp1.partition() -> Seq(100), tp2.partition() -> Seq(100), tp3.partition() -> Seq(100)),
servers = servers)
// Throttle to avoid race conditions
throttle(Seq(topicName), throttleSettingForSeconds(10), Map(
tp0 -> Seq(100, 101, 102),
tp1 -> Seq(100, 101, 102),
tp2 -> Seq(100, 101, 102),
tp3 -> Seq(100, 101, 102)
))
// API reassignment to 101 for tp0 and tp1
adminClient.alterPartitionReassignments(
Map(reassignmentEntry(tp0, Seq(101)), reassignmentEntry(tp1, Seq(101))).asJava
).all().get()
// cancel tp0, reassign tp1 to 102 (override), assign tp2 to 101 (new reassignment) and cancel tp3 (it is not moving)
val alterResults = adminClient.alterPartitionReassignments(
Map(cancelReassignmentEntry(tp0), reassignmentEntry(tp1, Seq(102)),
reassignmentEntry(tp2, Seq(101)), cancelReassignmentEntry(tp3)).asJava
).values()
alterResults.get(tp0).get()
alterResults.get(tp1).get()
alterResults.get(tp2).get()
try {
alterResults.get(tp3).get()
} catch {
case exception: Exception =>
assertEquals(exception.getCause.getClass, classOf[NoReassignmentInProgressException])
}
resetBrokersThrottle()
waitForAllReassignmentsToComplete()
assertEquals(Seq(100), zkClient.getReplicasForPartition(tp0))
assertEquals(Seq(102), zkClient.getReplicasForPartition(tp1))
assertEquals(Seq(101), zkClient.getReplicasForPartition(tp2))
assertEquals(Seq(100), zkClient.getReplicasForPartition(tp3))
}
/**
* Three different Alter Reassignment calls should all create reassignments
*/
@Test
def shouldBeAbleToIncrementallyStackDifferentReassignments(): Unit = {
startBrokers(Seq(100, 101))
adminClient = createAdminClient(servers)
val tpA0 = new TopicPartition("A", 0)
val tpA1 = new TopicPartition("A", 1)
val tpB0 = new TopicPartition("B", 0)
createTopic(zkClient, "A",
Map(tpA0.partition() -> Seq(100),
tpA1.partition() -> Seq(100)),
servers = servers)
createTopic(zkClient, "B",
Map(tpB0.partition() -> Seq(100)),
servers = servers)
// Throttle to avoid race conditions
throttle(Seq("A", "B"), throttleSettingForSeconds(10), Map(
tpA0 -> Seq(100, 101, 102),
tpA1 -> Seq(100, 101, 102),
tpB0 -> Seq(100, 101, 102)
))
adminClient.alterPartitionReassignments(Map(reassignmentEntry(tpA0, Seq(101))).asJava).all().get()
val apiReassignmentsInProgress1 = adminClient.listPartitionReassignments().reassignments().get()
assertEquals(1, apiReassignmentsInProgress1.size())
assertIsReassigning(
from = Seq(100), to = Seq(101),
apiReassignmentsInProgress1.get(tpA0)
)
adminClient.alterPartitionReassignments(Map(reassignmentEntry(tpA1, Seq(101))).asJava).all().get()
val apiReassignmentsInProgress2 = adminClient.listPartitionReassignments().reassignments().get()
assertEquals(2, apiReassignmentsInProgress2.size())
assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress2.get(tpA0))
assertIsReassigning(
from = Seq(100), to = Seq(101),
apiReassignmentsInProgress2.get(tpA1)
)
adminClient.alterPartitionReassignments(Map(reassignmentEntry(tpB0, Seq(101))).asJava).all().get()
val apiReassignmentsInProgress3 = adminClient.listPartitionReassignments().reassignments().get()
assertEquals(s"${apiReassignmentsInProgress3}", 3, apiReassignmentsInProgress3.size())
assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress3.get(tpA0))
assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress3.get(tpA1))
assertIsReassigning(
from = Seq(100), to = Seq(101),
apiReassignmentsInProgress3.get(tpB0)
)
resetBrokersThrottle()
waitForAllReassignmentsToComplete()
assertEquals(Seq(101), zkClient.getReplicasForPartition(tpA0))
assertEquals(Seq(101), zkClient.getReplicasForPartition(tpA1))
assertEquals(Seq(101), zkClient.getReplicasForPartition(tpB0))
}
/**
* Asserts that a replica is being reassigned from the given replicas to the target replicas
*/
def assertIsReassigning(from: Seq[Int], to: Seq[Int], reassignment: PartitionReassignment): Unit = {
assertReplicas((from ++ to).distinct, reassignment.replicas())
assertReplicas(to.filterNot(from.contains(_)), reassignment.addingReplicas())
assertReplicas(from.filterNot(to.contains(_)), reassignment.removingReplicas())
}
/**
* Asserts that a topic's reassignments completed and span across the expected replicas
*/
def assertMoveForTopicOccurred(expectedReplicas: Seq[Int],
partitionAssignments: Map[Int, PartitionReplicaAssignment]): Unit = {
assertEquals(expectedReplicas, partitionAssignments.values.flatMap(_.replicas).toSeq.distinct.sorted)
assertTrue(partitionAssignments.values.flatMap(_.addingReplicas).isEmpty)
assertTrue(partitionAssignments.values.flatMap(_.removingReplicas).isEmpty)
}
/**
* Asserts that a partition moved to the exact expected replicas in the specific order
*/
def assertMoveForPartitionOccurred(expectedReplicas: Seq[Int],
partitionAssignment: PartitionReplicaAssignment): Unit = {
assertEquals(expectedReplicas, partitionAssignment.replicas)
assertTrue(partitionAssignment.addingReplicas.isEmpty)
assertTrue(partitionAssignment.removingReplicas.isEmpty)
}
/**
* Asserts that two replica sets are equal, ignoring ordering
*/
def assertReplicas(expectedReplicas: Seq[Int], receivedReplicas: java.util.List[Integer]): Unit = {
assertEquals(expectedReplicas.sorted, receivedReplicas.asScala.map(_.toInt).sorted)
}
def throttleAllBrokersReplication(throttleBytes: String): Unit = {
val throttleConfigs = Seq(
new AlterConfigOp(new ConfigEntry(DynamicConfig.Broker.LeaderReplicationThrottledRateProp, throttleBytes), AlterConfigOp.OpType.SET),
new AlterConfigOp(new ConfigEntry(DynamicConfig.Broker.FollowerReplicationThrottledRateProp, throttleBytes), AlterConfigOp.OpType.SET)
).asJavaCollection
adminClient.incrementalAlterConfigs(
brokerIds.map { brokerId =>
new ConfigResource(ConfigResource.Type.BROKER, brokerId.toString) -> throttleConfigs
}.toMap.asJava
).all().get()
}
def resetBrokersThrottle(): Unit = throttleAllBrokersReplication(Int.MaxValue.toString)
def assignThrottledPartitionReplicas(allReplicasByPartition: Map[TopicPartition, Seq[Int]]): Unit = {
val throttles = allReplicasByPartition.groupBy(_._1.topic()).map {
case (topic, replicasByPartition) =>
new ConfigResource(ConfigResource.Type.TOPIC, topic) -> Seq(
new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, formatReplicaThrottles(replicasByPartition)), AlterConfigOp.OpType.SET),
new AlterConfigOp(new ConfigEntry(LogConfig.FollowerReplicationThrottledReplicasProp, formatReplicaThrottles(replicasByPartition)), AlterConfigOp.OpType.SET)
).asJavaCollection
}
adminClient.incrementalAlterConfigs(throttles.asJava).all().get()
}
def removePartitionReplicaThrottles(partitions: Set[TopicPartition]): Unit = {
val throttles = partitions.map {
tp =>
new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) -> Seq(
new AlterConfigOp(new ConfigEntry(LogConfig.LeaderReplicationThrottledReplicasProp, ""), AlterConfigOp.OpType.DELETE),
new AlterConfigOp(new ConfigEntry(LogConfig.FollowerReplicationThrottledReplicasProp, ""), AlterConfigOp.OpType.DELETE)
).asJavaCollection
}.toMap
adminClient.incrementalAlterConfigs(throttles.asJava).all().get()
}
def formatReplicaThrottles(moves: Map[TopicPartition, Seq[Int]]): String =
moves.flatMap { case (tp, assignment) =>
assignment.map(replicaId => s"${tp.partition}:$replicaId")
}.mkString(",")
def reassignmentEntry(tp: TopicPartition, replicas: Seq[Int]): (TopicPartition, java.util.Optional[NewPartitionReassignment]) =
tp -> java.util.Optional.of(new NewPartitionReassignment(replicas.map(_.asInstanceOf[Integer]).asJava))
def cancelReassignmentEntry(tp: TopicPartition): (TopicPartition, java.util.Optional[NewPartitionReassignment]) =
tp -> java.util.Optional.empty()
def waitForZkReassignmentToComplete(pause: Long = 100L): Unit = {
waitUntilTrue(() => !zkClient.reassignPartitionsInProgress,
s"Znode ${ReassignPartitionsZNode.path} wasn't deleted", pause = pause)
}
def json(topic: String*): String = {
def waitForAllReassignmentsToComplete(pause: Long = 100L): Unit = {
waitUntilTrue(() => adminClient.listPartitionReassignments().reassignments().get().isEmpty,
s"There still are ongoing reassignments", pause = pause)
}
def generateAssignmentJson(topic: String*): String = {
val topicStr = topic.map { t => "{\"topic\": \"" + t + "\"}" }.mkString(",")
s"""{"topics": [$topicStr],"version":1}"""
}
def executeAssignmentJson(partitions: Seq[PartitionAssignmentJson]): String =
s"""{"version":1,"partitions":[${partitions.map(_.toJson).mkString(",")}]}"""
case class PartitionAssignmentJson(topicPartition: TopicPartition, replicas: Seq[Int],
logDirectories: Option[Seq[String]] = None) {
def toJson: String = {
val logDirsSuffix = logDirectories match {
case Some(dirs) => s""","log_dirs":[${dirs.map("\"" + _ + "\"").mkString(",")}]"""
case None => ""
}
s"""{"topic":"${topicPartition.topic()}","partition":${topicPartition.partition()}""" +
s""","replicas":[${replicas.mkString(",")}]""" +
s"$logDirsSuffix}"
}
}
case class ThrottleSetting(throttleBytes: String, numMessages: Int, messageSizeBytes: Int)
def throttleSettingForSeconds(secondsDuration: Int): ThrottleSetting = {
val throttle = 1000 * 1000 // 1 MB/s throttle
val messageSize = 100 * 100 // 0.01 MB message size
val messagesPerSecond = throttle / messageSize
ThrottleSetting(throttle.toString, messagesPerSecond * secondsDuration, messageSize)
}
def throttle(topics: Seq[String], throttle: ThrottleSetting, replicasToThrottle: Map[TopicPartition, Seq[Int]]): Unit = {
val messagesPerTopic = throttle.numMessages / topics.size
for (topic <- topics) {
produceMessages(topic, numMessages = messagesPerTopic, acks = 0, valueLength = throttle.messageSizeBytes)
}
throttleAllBrokersReplication(throttle.throttleBytes)
assignThrottledPartitionReplicas(replicasToThrottle)
}
private def produceMessages(topic: String, numMessages: Int, acks: Int, valueLength: Int): Unit = {
val records = (0 until numMessages).map(_ => new ProducerRecord[Array[Byte], Array[Byte]](topic,
new Array[Byte](valueLength)))

View File

@ -58,7 +58,7 @@ class ControllerChannelManagerTest {
partitions.foreach { case (partition, leaderAndIsr) =>
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
context.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, Seq(1, 2, 3), isNew = false)
batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(1, 2, 3)), isNew = false)
}
batch.sendRequestsToBrokers(controllerEpoch)
@ -96,8 +96,8 @@ class ControllerChannelManagerTest {
context.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
batch.newBatch()
batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, Seq(1, 2, 3), isNew = true)
batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, Seq(1, 2, 3), isNew = false)
batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(1, 2, 3)), isNew = true)
batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(1, 2, 3)), isNew = false)
batch.sendRequestsToBrokers(controllerEpoch)
val leaderAndIsrRequests = batch.collectLeaderAndIsrRequestsFor(2)
@ -126,7 +126,7 @@ class ControllerChannelManagerTest {
context.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
batch.newBatch()
batch.addLeaderAndIsrRequestForBrokers(Seq(1, 2, 3), partition, leaderIsrAndControllerEpoch, Seq(1, 2, 3), isNew = false)
batch.addLeaderAndIsrRequestForBrokers(Seq(1, 2, 3), partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(1, 2, 3)), isNew = false)
batch.sendRequestsToBrokers(controllerEpoch)
assertEquals(0, batch.sentEvents.size)
@ -170,7 +170,7 @@ class ControllerChannelManagerTest {
context.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
batch.newBatch()
batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, Seq(1, 2, 3), isNew = false)
batch.addLeaderAndIsrRequestForBrokers(Seq(2), partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(1, 2, 3)), isNew = false)
batch.sendRequestsToBrokers(controllerEpoch)
val leaderAndIsrRequests = batch.collectLeaderAndIsrRequestsFor(2, expectedLeaderAndIsrVersion)
@ -635,6 +635,8 @@ class ControllerChannelManagerTest {
KafkaConfig.fromProps(props)
}
private def replicaAssignment(replicas: Seq[Int]): PartitionReplicaAssignment = PartitionReplicaAssignment(replicas, Seq(), Seq())
private def initContext(brokers: Seq[Int],
topics: Set[String],
numPartitions: Int,

View File

@ -0,0 +1,189 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package unit.kafka.controller
import kafka.cluster.{Broker, EndPoint}
import kafka.controller.{ControllerContext, PartitionReplicaAssignment}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.junit.{Before, Test}
import org.junit.Assert.assertEquals
import org.junit.Assert.assertTrue
import org.junit.Assert.assertFalse
class ControllerContextTest {
var context: ControllerContext = null
val brokers: Seq[Int] = Seq(1, 2, 3)
val tp1 = new TopicPartition("A", 0)
val tp2 = new TopicPartition("A", 1)
val tp3 = new TopicPartition("B", 0)
@Before
def setUp(): Unit = {
context = new ControllerContext
val brokerEpochs = Seq(1,2,3).map { brokerId =>
val endpoint = new EndPoint("localhost", 9900 + brokerId, new ListenerName("PLAINTEXT"),
SecurityProtocol.PLAINTEXT)
Broker(brokerId, Seq(endpoint), rack = None) -> 1L
}.toMap
context.setLiveBrokerAndEpochs(brokerEpochs)
// Simple round-robin replica assignment
var leaderIndex = 0
Seq(tp1, tp2, tp3).foreach {
partition =>
val replicas = brokers.indices.map { i =>
val replica = brokers((i + leaderIndex) % brokers.size)
replica
}
context.updatePartitionReplicaAssignment(partition, replicas)
leaderIndex += 1
}
}
@Test
def testUpdatePartitionReplicaAssignmentUpdatesReplicaAssignmentOnly(): Unit = {
val expectedReplicas = Seq(4)
context.updatePartitionReplicaAssignment(tp1, expectedReplicas)
val assignment = context.partitionReplicaAssignment(tp1)
val fullAssignment = context.partitionFullReplicaAssignment(tp1)
assertEquals(expectedReplicas, assignment)
assertEquals(expectedReplicas, fullAssignment.replicas)
assertEquals(Seq(), fullAssignment.addingReplicas)
assertEquals(Seq(), fullAssignment.removingReplicas)
}
@Test
def testUpdatePartitionReplicaAssignmentUpdatesReplicaAssignmentOnlyAndDoesNotOverwrite(): Unit = {
val expectedReplicas = Seq(4)
val expectedFullAssignment = PartitionReplicaAssignment(Seq(3), Seq(1), Seq(2))
context.updatePartitionFullReplicaAssignment(tp1, expectedFullAssignment)
context.updatePartitionReplicaAssignment(tp1, expectedReplicas) // update only the replicas
val assignment = context.partitionReplicaAssignment(tp1)
val fullAssignment = context.partitionFullReplicaAssignment(tp1)
assertEquals(expectedReplicas, assignment)
assertEquals(expectedReplicas, fullAssignment.replicas)
// adding/removing replicas preserved
assertEquals(Seq(1), fullAssignment.addingReplicas)
assertEquals(Seq(2), fullAssignment.removingReplicas)
}
@Test
def testUpdatePartitionFullReplicaAssignmentUpdatesReplicaAssignment(): Unit = {
val initialReplicas = Seq(4)
context.updatePartitionReplicaAssignment(tp1, initialReplicas) // update only the replicas
val fullAssignment = context.partitionFullReplicaAssignment(tp1)
assertEquals(initialReplicas, fullAssignment.replicas)
assertEquals(Seq(), fullAssignment.addingReplicas)
assertEquals(Seq(), fullAssignment.removingReplicas)
val expectedFullAssignment = PartitionReplicaAssignment(Seq(3), Seq(1), Seq(2))
context.updatePartitionFullReplicaAssignment(tp1, expectedFullAssignment)
val updatedFullAssignment = context.partitionFullReplicaAssignment(tp1)
assertEquals(expectedFullAssignment.replicas, updatedFullAssignment.replicas)
assertEquals(expectedFullAssignment.addingReplicas, updatedFullAssignment.addingReplicas)
assertEquals(expectedFullAssignment.removingReplicas, updatedFullAssignment.removingReplicas)
}
@Test
def testPartitionReplicaAssignmentReturnsEmptySeqIfTopicOrPartitionDoesNotExist(): Unit = {
val noTopicReplicas = context.partitionReplicaAssignment(new TopicPartition("NONEXISTENT", 0))
assertEquals(Seq.empty, noTopicReplicas)
val noPartitionReplicas = context.partitionReplicaAssignment(new TopicPartition("A", 100))
assertEquals(Seq.empty, noPartitionReplicas)
}
@Test
def testPartitionFullReplicaAssignmentReturnsEmptyAssignmentIfTopicOrPartitionDoesNotExist(): Unit = {
val expectedEmptyAssignment = PartitionReplicaAssignment(Seq.empty, Seq.empty, Seq.empty)
val noTopicAssignment = context.partitionFullReplicaAssignment(new TopicPartition("NONEXISTENT", 0))
assertEquals(expectedEmptyAssignment, noTopicAssignment)
val noPartitionAssignment = context.partitionFullReplicaAssignment(new TopicPartition("A", 100))
assertEquals(expectedEmptyAssignment, noPartitionAssignment)
}
@Test
def testPartitionReplicaAssignmentForTopicReturnsEmptyMapIfTopicDoesNotExist(): Unit = {
assertEquals(Map.empty, context.partitionReplicaAssignmentForTopic("NONEXISTENT"))
}
@Test
def testPartitionReplicaAssignmentForTopicReturnsExpectedReplicaAssignments(): Unit = {
val expectedAssignments = Map(
tp1 -> context.partitionReplicaAssignment(tp1),
tp2 -> context.partitionReplicaAssignment(tp2)
)
val receivedAssignments = context.partitionReplicaAssignmentForTopic("A")
assertEquals(expectedAssignments, receivedAssignments)
}
@Test
def testPartitionReplicaAssignment(): Unit = {
val reassigningPartition = PartitionReplicaAssignment(List(1, 2, 3, 4, 5, 6), List(2, 3, 4), List(1, 5, 6))
assertTrue(reassigningPartition.isBeingReassigned)
assertEquals(List(2, 3, 4), reassigningPartition.targetReplicas)
val reassigningPartition2 = PartitionReplicaAssignment(List(1, 2, 3, 4), List(), List(1, 4))
assertTrue(reassigningPartition2.isBeingReassigned)
assertEquals(List(2, 3), reassigningPartition2.targetReplicas)
val reassigningPartition3 = PartitionReplicaAssignment(List(1, 2, 3, 4), List(4), List(2))
assertTrue(reassigningPartition3.isBeingReassigned)
assertEquals(List(1, 3, 4), reassigningPartition3.targetReplicas)
val partition = PartitionReplicaAssignment(List(1, 2, 3, 4, 5, 6), List(), List())
assertFalse(partition.isBeingReassigned)
assertEquals(List(1, 2, 3, 4, 5, 6), partition.targetReplicas)
val reassigningPartition4 = PartitionReplicaAssignment.fromOldAndNewReplicas(
List(1, 2, 3, 4), List(4, 2, 5, 3)
)
assertEquals(List(4, 2, 5, 3, 1), reassigningPartition4.replicas)
assertEquals(List(4, 2, 5, 3), reassigningPartition4.targetReplicas)
assertEquals(List(5), reassigningPartition4.addingReplicas)
assertEquals(List(1), reassigningPartition4.removingReplicas)
assertTrue(reassigningPartition4.isBeingReassigned)
val reassigningPartition5 = PartitionReplicaAssignment.fromOldAndNewReplicas(
List(1, 2, 3), List(4, 5, 6)
)
assertEquals(List(4, 5, 6, 1, 2, 3), reassigningPartition5.replicas)
assertEquals(List(4, 5, 6), reassigningPartition5.targetReplicas)
assertEquals(List(4, 5, 6), reassigningPartition5.addingReplicas)
assertEquals(List(1, 2, 3), reassigningPartition5.removingReplicas)
assertTrue(reassigningPartition5.isBeingReassigned)
val nonReassigningPartition = PartitionReplicaAssignment.fromOldAndNewReplicas(
List(1, 2, 3), List(3, 1, 2)
)
assertEquals(List(3, 1, 2), nonReassigningPartition.replicas)
assertEquals(List(3, 1, 2), nonReassigningPartition.targetReplicas)
assertEquals(List(), nonReassigningPartition.addingReplicas)
assertEquals(List(), nonReassigningPartition.removingReplicas)
assertFalse(nonReassigningPartition.isBeingReassigned)
}
}

View File

@ -249,7 +249,9 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
val tp0 = new TopicPartition("t", 0)
val tp1 = new TopicPartition("t", 1)
val assignment = Map(tp0.partition -> Seq(0))
val expandedAssignment = Map(tp0 -> Seq(0), tp1 -> Seq(0))
val expandedAssignment = Map(
tp0 -> PartitionReplicaAssignment(Seq(0), Seq(), Seq()),
tp1 -> PartitionReplicaAssignment(Seq(0), Seq(), Seq()))
TestUtils.createTopic(zkClient, tp0.topic, partitionReplicaAssignment = assignment, servers = servers)
zkClient.setTopicAssignment(tp0.topic, expandedAssignment, firstControllerEpochZkVersion)
waitForPartitionState(tp1, firstControllerEpoch, 0, LeaderAndIsr.initialLeaderEpoch,
@ -265,7 +267,9 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
val tp0 = new TopicPartition("t", 0)
val tp1 = new TopicPartition("t", 1)
val assignment = Map(tp0.partition -> Seq(otherBrokerId, controllerId))
val expandedAssignment = Map(tp0 -> Seq(otherBrokerId, controllerId), tp1 -> Seq(otherBrokerId, controllerId))
val expandedAssignment = Map(
tp0 -> PartitionReplicaAssignment(Seq(otherBrokerId, controllerId), Seq(), Seq()),
tp1 -> PartitionReplicaAssignment(Seq(otherBrokerId, controllerId), Seq(), Seq()))
TestUtils.createTopic(zkClient, tp0.topic, partitionReplicaAssignment = assignment, servers = servers)
servers(otherBrokerId).shutdown()
servers(otherBrokerId).awaitShutdown()
@ -280,18 +284,18 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
servers = makeServers(2)
val controllerId = TestUtils.waitUntilControllerElected(zkClient)
val metricName = s"kafka.controller:type=ControllerStats,name=${ControllerState.PartitionReassignment.rateAndTimeMetricName.get}"
val metricName = s"kafka.controller:type=ControllerStats,name=${ControllerState.AlterPartitionReassignment.rateAndTimeMetricName.get}"
val timerCount = timer(metricName).count
val otherBrokerId = servers.map(_.config.brokerId).filter(_ != controllerId).head
val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(controllerId))
val reassignment = Map(tp -> Seq(otherBrokerId))
val reassignment = Map(tp -> PartitionReplicaAssignment(Seq(otherBrokerId), List(), List()))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
zkClient.createPartitionReassignment(reassignment)
zkClient.createPartitionReassignment(reassignment.mapValues(_.replicas).toMap)
waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.initialLeaderEpoch + 3,
"failed to get expected partition state after partition reassignment")
TestUtils.waitUntilTrue(() => zkClient.getReplicaAssignmentForTopics(Set(tp.topic)) == reassignment,
TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment,
"failed to get updated partition assignment on topic znode after partition reassignment")
TestUtils.waitUntilTrue(() => !zkClient.reassignPartitionsInProgress(),
"failed to remove reassign partitions path after completion")
@ -326,17 +330,17 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
val otherBrokerId = servers.map(_.config.brokerId).filter(_ != controllerId).head
val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(controllerId))
val reassignment = Map(tp -> Seq(otherBrokerId))
val reassignment = Map(tp -> PartitionReplicaAssignment(Seq(otherBrokerId), List(), List()))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
servers(otherBrokerId).shutdown()
servers(otherBrokerId).awaitShutdown()
zkClient.createPartitionReassignment(reassignment)
zkClient.createPartitionReassignment(reassignment.mapValues(_.replicas).toMap)
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.initialLeaderEpoch + 1,
"failed to get expected partition state during partition reassignment with offline replica")
servers(otherBrokerId).startup()
waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.initialLeaderEpoch + 4,
"failed to get expected partition state after partition reassignment")
TestUtils.waitUntilTrue(() => zkClient.getReplicaAssignmentForTopics(Set(tp.topic)) == reassignment,
TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment,
"failed to get updated partition assignment on topic znode after partition reassignment")
TestUtils.waitUntilTrue(() => !zkClient.reassignPartitionsInProgress(),
"failed to remove reassign partitions path after completion")

View File

@ -89,7 +89,7 @@ class PartitionStateMachineTest {
EasyMock.expect(mockZkClient.createTopicPartitionStatesRaw(Map(partition -> leaderIsrAndControllerEpoch), controllerContext.epochZkVersion))
.andReturn(Seq(CreateResponse(Code.OK, null, Some(partition), null, ResponseMetadata(0, 0))))
EasyMock.expect(mockControllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(brokerId),
partition, leaderIsrAndControllerEpoch, Seq(brokerId), isNew = true))
partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(brokerId)), isNew = true))
EasyMock.expect(mockControllerBrokerRequestBatch.sendRequestsToBrokers(controllerEpoch))
EasyMock.replay(mockZkClient, mockControllerBrokerRequestBatch)
partitionStateMachine.handleStateChanges(
@ -175,7 +175,7 @@ class PartitionStateMachineTest {
EasyMock.expect(mockZkClient.updateLeaderAndIsr(Map(partition -> leaderAndIsrAfterElection), controllerEpoch, controllerContext.epochZkVersion))
.andReturn(UpdateLeaderAndIsrResult(Map(partition -> Right(updatedLeaderAndIsr)), Seq.empty))
EasyMock.expect(mockControllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(brokerId),
partition, LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch), Seq(brokerId), isNew = false))
partition, LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch), replicaAssignment(Seq(brokerId)), isNew = false))
EasyMock.expect(mockControllerBrokerRequestBatch.sendRequestsToBrokers(controllerEpoch))
EasyMock.replay(mockZkClient, mockControllerBrokerRequestBatch)
@ -210,7 +210,7 @@ class PartitionStateMachineTest {
// The leaderAndIsr request should be sent to both brokers, including the shutting down one
EasyMock.expect(mockControllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(brokerId, otherBrokerId),
partition, LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch), Seq(brokerId, otherBrokerId),
partition, LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch), replicaAssignment(Seq(brokerId, otherBrokerId)),
isNew = false))
EasyMock.expect(mockControllerBrokerRequestBatch.sendRequestsToBrokers(controllerEpoch))
EasyMock.replay(mockZkClient, mockControllerBrokerRequestBatch)
@ -263,7 +263,7 @@ class PartitionStateMachineTest {
EasyMock.expect(mockZkClient.updateLeaderAndIsr(Map(partition -> leaderAndIsrAfterElection), controllerEpoch, controllerContext.epochZkVersion))
.andReturn(UpdateLeaderAndIsrResult(Map(partition -> Right(updatedLeaderAndIsr)), Seq.empty))
EasyMock.expect(mockControllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(brokerId),
partition, LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch), Seq(brokerId), isNew = false))
partition, LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch), replicaAssignment(Seq(brokerId)), isNew = false))
EasyMock.expect(mockControllerBrokerRequestBatch.sendRequestsToBrokers(controllerEpoch))
EasyMock.replay(mockZkClient, mockControllerBrokerRequestBatch)
@ -327,7 +327,7 @@ class PartitionStateMachineTest {
Seq(brokerId),
partition,
LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch),
Seq(leaderBrokerId, brokerId),
replicaAssignment(Seq(leaderBrokerId, brokerId)),
false
)
)
@ -516,4 +516,7 @@ class PartitionStateMachineTest {
topicDeletionManager.enqueueTopicsForDeletion(Set(topic))
assertEquals(s"There should be no offline partition(s)", 0, controllerContext.offlinePartitionCount)
}
private def replicaAssignment(replicas: Seq[Int]): PartitionReplicaAssignment = PartitionReplicaAssignment(replicas, Seq(), Seq())
}

View File

@ -194,7 +194,7 @@ class ReplicaStateMachineTest {
controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
EasyMock.expect(mockControllerBrokerRequestBatch.newBatch())
EasyMock.expect(mockControllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(brokerId),
partition, leaderIsrAndControllerEpoch, Seq(brokerId), isNew = false))
partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(brokerId)), isNew = false))
EasyMock.expect(mockControllerBrokerRequestBatch.sendRequestsToBrokers(controllerEpoch))
EasyMock.replay(mockZkClient, mockControllerBrokerRequestBatch)
replicaStateMachine.handleStateChanges(replicas, OnlineReplica)
@ -224,7 +224,7 @@ class ReplicaStateMachineTest {
EasyMock.expect(mockZkClient.updateLeaderAndIsr(Map(partition -> adjustedLeaderAndIsr), controllerEpoch, controllerContext.epochZkVersion))
.andReturn(UpdateLeaderAndIsrResult(Map(partition -> Right(updatedLeaderAndIsr)), Seq.empty))
EasyMock.expect(mockControllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(otherBrokerId),
partition, updatedLeaderIsrAndControllerEpoch, replicaIds, isNew = false))
partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment(replicaIds), isNew = false))
EasyMock.expect(mockControllerBrokerRequestBatch.sendRequestsToBrokers(controllerEpoch))
EasyMock.replay(mockZkClient, mockControllerBrokerRequestBatch)
@ -267,7 +267,7 @@ class ReplicaStateMachineTest {
controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
EasyMock.expect(mockControllerBrokerRequestBatch.newBatch())
EasyMock.expect(mockControllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(brokerId),
partition, leaderIsrAndControllerEpoch, Seq(brokerId), isNew = false))
partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(brokerId)), isNew = false))
EasyMock.expect(mockControllerBrokerRequestBatch.sendRequestsToBrokers(controllerEpoch))
EasyMock.replay(mockZkClient, mockControllerBrokerRequestBatch)
replicaStateMachine.handleStateChanges(replicas, OnlineReplica)
@ -385,7 +385,7 @@ class ReplicaStateMachineTest {
controllerContext.partitionLeadershipInfo.put(partition, leaderIsrAndControllerEpoch)
EasyMock.expect(mockControllerBrokerRequestBatch.newBatch())
EasyMock.expect(mockControllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(Seq(brokerId),
partition, leaderIsrAndControllerEpoch, Seq(brokerId), isNew = false))
partition, leaderIsrAndControllerEpoch, replicaAssignment(Seq(brokerId)), isNew = false))
EasyMock.expect(mockControllerBrokerRequestBatch.sendRequestsToBrokers(controllerEpoch))
EasyMock.replay(mockZkClient, mockControllerBrokerRequestBatch)
replicaStateMachine.handleStateChanges(replicas, OnlineReplica)
@ -408,4 +408,7 @@ class ReplicaStateMachineTest {
replicaStateMachine.handleStateChanges(replicas, toState)
assertEquals(fromState, replicaState(replica))
}
private def replicaAssignment(replicas: Seq[Int]): PartitionReplicaAssignment = PartitionReplicaAssignment(replicas, Seq(), Seq())
}

View File

@ -32,6 +32,7 @@ import scala.util.{Failure, Success, Try}
import javax.security.auth.login.Configuration
import kafka.api.ApiVersion
import kafka.cluster.{Broker, EndPoint}
import kafka.controller.PartitionReplicaAssignment
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Time
@ -130,7 +131,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging {
// Test that can update persistent nodes
val updatedAssignment = assignment - new TopicPartition(topic1, 2)
zkClient.setTopicAssignment(topic1, updatedAssignment)
zkClient.setTopicAssignment(topic1, updatedAssignment.mapValues { case (v) => PartitionReplicaAssignment(v, List(), List()) }.toMap)
assertEquals(updatedAssignment.size, zkClient.getTopicPartitionCount(topic1).get)
}

View File

@ -19,6 +19,7 @@ package kafka.admin
import java.util
import java.util.Properties
import kafka.controller.PartitionReplicaAssignment
import kafka.log._
import kafka.server.DynamicConfig.Broker._
import kafka.server.KafkaConfig._
@ -87,7 +88,7 @@ class AdminZkClientTest extends ZooKeeperTestHarness with Logging with RackAware
1 -> List(1, 2, 3))
adminZkClient.createTopicWithAssignment("test", topicConfig, assignment)
val found = zkClient.getPartitionAssignmentForTopics(Set("test"))
assertEquals(assignment, found("test"))
assertEquals(assignment.mapValues(PartitionReplicaAssignment(_, List(), List())).toMap, found("test"))
}
@Test
@ -179,8 +180,9 @@ class AdminZkClientTest extends ZooKeeperTestHarness with Logging with RackAware
catch { case _: TopicExistsException => () }
val (_, partitionAssignment) = zkClient.getPartitionAssignmentForTopics(Set(topic)).head
assertEquals(3, partitionAssignment.size)
partitionAssignment.foreach { case (partition, replicas) =>
assertEquals(s"Unexpected replication factor for $partition", 1, replicas.size)
partitionAssignment.foreach { case (partition, partitionReplicaAssignment) =>
assertEquals(s"Unexpected replication factor for $partition",
1, partitionReplicaAssignment.replicas.size)
}
val savedProps = zkClient.getEntityConfigs(ConfigType.Topic, topic)
assertEquals(props, savedProps)

View File

@ -40,7 +40,7 @@ import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import scala.collection.{Seq, mutable}
import scala.util.Random
import kafka.controller.LeaderIsrAndControllerEpoch
import kafka.controller.{LeaderIsrAndControllerEpoch, PartitionReplicaAssignment}
import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult
import kafka.zookeeper._
import org.apache.kafka.common.errors.ControllerMovedException
@ -169,7 +169,7 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
val expectedAssignment = assignment map { topicAssignment =>
val partition = topicAssignment._1.partition
val assignment = topicAssignment._2
partition -> assignment
partition -> PartitionReplicaAssignment(assignment, List(), List())
}
assertEquals(assignment.size, zkClient.getTopicPartitionCount(topic1).get)
@ -179,7 +179,7 @@ class KafkaZkClientTest extends ZooKeeperTestHarness {
val updatedAssignment = assignment - new TopicPartition(topic1, 2)
zkClient.setTopicAssignment(topic1, updatedAssignment)
zkClient.setTopicAssignment(topic1, updatedAssignment.mapValues { case v => PartitionReplicaAssignment(v, List(), List()) }.toMap)
assertEquals(updatedAssignment.size, zkClient.getTopicPartitionCount(topic1).get)
// add second topic

View File

@ -21,6 +21,8 @@
<h5><a id="upgrade_240_notable" href="#upgrade_240_notable">Notable changes in 2.4.0</a></h5>
<ul>
<li>A new Admin API has been added for partition reassignments. Due to changing the way Kafka propagates reassignment information,
it is possible to lose reassignment state in failure edge cases while upgrading to the new version. It is not recommended to start reassignments while upgrading.</li>
<li>ZooKeeper has been upgraded from 3.4.14 to 3.5.5. TLS and dynamic reconfiguration are supported by the new version.</li>
<li>The <code>bin/kafka-preferred-replica-election.sh</code> command line tool has been deprecated. It has been replaced by <code>bin/kafka-leader-election.sh</code>.</li>
<li>The methods <code>electPreferredLeaders</code> in the Java <code>AdminClient</code> class have been deprecated in favor of the methods <code>electLeaders</code>.</li>