mirror of https://github.com/apache/kafka.git
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:
parent
41b89a6ecf
commit
18d4e57f6e
|
@ -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)
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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]],
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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}")
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)))
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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")
|
||||
|
|
|
@ -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())
|
||||
|
||||
}
|
||||
|
|
|
@ -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())
|
||||
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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>
|
||||
|
|
Loading…
Reference in New Issue