You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/09/18 05:58:45 UTC

[GitHub] [kafka] bob-barrett opened a new pull request #9302: KAFKA-10149: Allow auto preferred leader election when partitions are reassigning

bob-barrett opened a new pull request #9302:
URL: https://github.com/apache/kafka/pull/9302


   This patch removes the check for reassigning partitions when determining whether to trigger automatic leader election. This check can cause problems during long-running reassignments because a crashed broker can leave the partition leaderships in an unexpectedly unbalanced state. 
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #9302: KAFKA-10149: Allow auto preferred leader election when partitions are reassigning

Posted by GitBox <gi...@apache.org>.
dajac commented on a change in pull request #9302:
URL: https://github.com/apache/kafka/pull/9302#discussion_r495068017



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1067,10 +1067,7 @@ class KafkaController(val config: KafkaConfig,
       // check ratio and if greater than desired ratio, trigger a rebalance for the topic partitions
       // that need to be on this broker
       if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) {
-        // do this check only if the broker is live and there are no partitions being reassigned currently
-        // and preferred replica election is not in progress
         val candidatePartitions = topicsNotInPreferredReplica.keys.filter(tp =>
-          controllerContext.partitionsBeingReassigned.isEmpty &&

Review comment:
       I totally agree. The sooner the better. I was wondering if there could be any conflicts with the reassignment logic. I have looked at it and I haven't seen anything. 👍 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] hachikuji commented on a change in pull request #9302: KAFKA-10149: Allow auto preferred leader election when partitions are reassigning

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9302:
URL: https://github.com/apache/kafka/pull/9302#discussion_r491094781



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1067,10 +1067,7 @@ class KafkaController(val config: KafkaConfig,
       // check ratio and if greater than desired ratio, trigger a rebalance for the topic partitions
       // that need to be on this broker
       if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) {
-        // do this check only if the broker is live and there are no partitions being reassigned currently
-        // and preferred replica election is not in progress
         val candidatePartitions = topicsNotInPreferredReplica.keys.filter(tp =>
-          controllerContext.partitionsBeingReassigned.isEmpty &&

Review comment:
       It's a good question. During a reassignment, the adding replicas are always listed first which means the preferred leader is among the target replicas. My take is that we want to move the leadership onto the new preferred leader as soon as possible since the whole point of the reassignment is to take load off the removing replicas.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #9302: KAFKA-10149: Allow auto preferred leader election when partitions are reassigning

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #9302:
URL: https://github.com/apache/kafka/pull/9302#issuecomment-854395748


   Is this still relevant?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] hachikuji commented on a change in pull request #9302: KAFKA-10149: Allow auto preferred leader election when partitions are reassigning

Posted by GitBox <gi...@apache.org>.
hachikuji commented on a change in pull request #9302:
URL: https://github.com/apache/kafka/pull/9302#discussion_r491099373



##########
File path: core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala
##########
@@ -444,6 +444,29 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness {
       "failed to get expected partition state upon broker startup")
   }
 
+  @Test
+  def testAutoPreferredReplicaLeaderElectionWithReassigningPartitions(): Unit = {
+    servers = makeServers(2, autoLeaderRebalanceEnable = true)
+    val controllerId = TestUtils.waitUntilControllerElected(zkClient)
+    val otherBrokerId = servers.map(_.config.brokerId).filter(_ != controllerId).head
+    val tp = new TopicPartition("t", 0)
+    val assignment = Map(tp.partition -> Seq(1, 0))
+    TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
+    val reassigningTp = new TopicPartition("reassigning", 0)
+    val reassigningAssignment = Map(reassigningTp.partition -> Seq(0))
+    TestUtils.createTopic(zkClient, reassigningTp.topic, partitionReplicaAssignment = reassigningAssignment, servers = servers)
+    servers(otherBrokerId).shutdown()
+    servers(otherBrokerId).awaitShutdown()
+    waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.initialLeaderEpoch + 1,
+      "failed to get expected partition state upon broker shutdown")
+
+    // Directly edit the controller context to simulate a reassigning partition
+    servers(controllerId).kafkaController.controllerContext.partitionsBeingReassigned.add(reassigningTp)

Review comment:
       Feels a bit nasty to dip into the controller context directly. An alternative way of testing this might be something like this:
   
   1. Start with two partitions
   2. Reassign one partition to a new replica with a low throttle
   3. Reassign the other partition so that just the preferred leader changes
   4. Trigger preferred election and verify that the leader of the second partition has changed
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] dajac commented on a change in pull request #9302: KAFKA-10149: Allow auto preferred leader election when partitions are reassigning

Posted by GitBox <gi...@apache.org>.
dajac commented on a change in pull request #9302:
URL: https://github.com/apache/kafka/pull/9302#discussion_r490794812



##########
File path: core/src/main/scala/kafka/controller/KafkaController.scala
##########
@@ -1067,10 +1067,7 @@ class KafkaController(val config: KafkaConfig,
       // check ratio and if greater than desired ratio, trigger a rebalance for the topic partitions
       // that need to be on this broker
       if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) {
-        // do this check only if the broker is live and there are no partitions being reassigned currently
-        // and preferred replica election is not in progress
         val candidatePartitions = topicsNotInPreferredReplica.keys.filter(tp =>
-          controllerContext.partitionsBeingReassigned.isEmpty &&

Review comment:
       @bob-barrett It totally makes sense to not block it the auto preferred leader election if there is any ongoing reassignments in the cluster. However, I wonder if we should continue to disallow it for the partitions being reassigned in order to not mess up with the their ongoing reassignment. I am not sure if this is a real concern or not though. Have you thought about this case?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #9302: KAFKA-10149: Allow auto preferred leader election when partitions are reassigning

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #9302:
URL: https://github.com/apache/kafka/pull/9302#issuecomment-854395748


   Is this still relevant?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org