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 2021/04/19 02:42:22 UTC

[GitHub] [kafka] wenbingshen opened a new pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

wenbingshen opened a new pull request #10558:
URL: https://github.com/apache/kafka/pull/10558


   When using the kafka-election-tool for preferred replica election, if there are partitions in the elected list that are in the preferred replica, the list of partitions already in the preferred replica will be replaced by the successfully elected partition list.
   
   ### 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 pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

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


   @wenbingshen Thanks for the patch. Should we add a unit test to verify this or perhaps extend an existing one?


-- 
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] chia7712 commented on pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

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


   @wenbingshen thanks for the updated code. I will merge it after QA pass.


-- 
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] wenbingshen commented on a change in pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

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



##########
File path: core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala
##########
@@ -273,6 +273,48 @@ final class LeaderElectionCommandTest extends ZooKeeperTestHarness {
     ))
     assertTrue(e.getCause.isInstanceOf[TimeoutException])
   }
+
+  @Test
+  def testElectionResultOutput(): Unit = {
+    TestUtils.resource(Admin.create(createConfig(servers).asJava)) { client =>
+      val topic = "non-preferred-topic"
+      val partition0 = 0
+      val partition1 = 1
+      val assignment0 = Seq(broker2, broker3)
+      val assignment1 = Seq(broker3, broker2)
+
+      TestUtils.createTopic(zkClient, topic, Map(partition0 -> assignment0, partition1 -> assignment1), servers)
+
+      val topicPartition0 = new TopicPartition(topic, partition0)
+      val topicPartition1 = new TopicPartition(topic, partition1)
+
+      TestUtils.assertLeader(client, topicPartition0, broker2)
+      TestUtils.assertLeader(client, topicPartition1, broker3)
+
+      servers(broker2).shutdown()
+      TestUtils.assertLeader(client, topicPartition0, broker3)
+      servers(broker2).startup()
+      TestUtils.waitForBrokersInIsr(client, topicPartition0, Set(broker2))
+      TestUtils.waitForBrokersInIsr(client, topicPartition1, Set(broker2))
+
+      val topicPartitionPath = tempTopicPartitionFile(Set(topicPartition0, topicPartition1))
+      val output = TestUtils.grabConsoleOutput(
+        LeaderElectionCommand.main(
+          Array(
+            "--bootstrap-server", bootstrapServers(servers),
+            "--election-type", "preferred",
+            "--path-to-json-file", topicPartitionPath.toString
+          )
+        )
+      )
+
+      val electionResultOutputIter = output.split("\n").iterator
+      assertTrue(electionResultOutputIter.hasNext)
+      assertTrue(electionResultOutputIter.next().contains(s"Successfully completed leader election (PREFERRED) for partitions $topicPartition0"))
+      assertTrue(electionResultOutputIter.hasNext)
+      assertTrue(electionResultOutputIter.next().contains(s"Valid replica already elected for partitions $topicPartition1"))

Review comment:
       Dear @dajac Following your suggestion, I added a unit test to verify the output of noop set and succeeded set. Are you satisfied with this? :)




-- 
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] wenbingshen commented on pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

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


   @dajac @chia7712  Can you help review it? Thanks. :)


-- 
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] wenbingshen commented on a change in pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

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



##########
File path: core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala
##########
@@ -273,6 +273,48 @@ final class LeaderElectionCommandTest extends ZooKeeperTestHarness {
     ))
     assertTrue(e.getCause.isInstanceOf[TimeoutException])
   }
+
+  @Test
+  def testElectionResultOutput(): Unit = {
+    TestUtils.resource(Admin.create(createConfig(servers).asJava)) { client =>
+      val topic = "non-preferred-topic"
+      val partition0 = 0
+      val partition1 = 1
+      val assignment0 = Seq(broker2, broker3)
+      val assignment1 = Seq(broker3, broker2)
+
+      TestUtils.createTopic(zkClient, topic, Map(partition0 -> assignment0, partition1 -> assignment1), servers)
+
+      val topicPartition0 = new TopicPartition(topic, partition0)
+      val topicPartition1 = new TopicPartition(topic, partition1)
+
+      TestUtils.assertLeader(client, topicPartition0, broker2)
+      TestUtils.assertLeader(client, topicPartition1, broker3)
+
+      servers(broker2).shutdown()
+      TestUtils.assertLeader(client, topicPartition0, broker3)
+      servers(broker2).startup()
+      TestUtils.waitForBrokersInIsr(client, topicPartition0, Set(broker2))
+      TestUtils.waitForBrokersInIsr(client, topicPartition1, Set(broker2))
+
+      val topicPartitionPath = tempTopicPartitionFile(Set(topicPartition0, topicPartition1))
+      val output = TestUtils.grabConsoleOutput(
+        LeaderElectionCommand.main(
+          Array(
+            "--bootstrap-server", bootstrapServers(servers),
+            "--election-type", "preferred",
+            "--path-to-json-file", topicPartitionPath.toString
+          )
+        )
+      )
+
+      val electionResultOutputIter = output.split("\n").iterator
+      assertTrue(electionResultOutputIter.hasNext)
+      assertTrue(electionResultOutputIter.next().contains(s"Successfully completed leader election (PREFERRED) for partitions $topicPartition0"))
+      assertTrue(electionResultOutputIter.hasNext)
+      assertTrue(electionResultOutputIter.next().contains(s"Valid replica already elected for partitions $topicPartition1"))

Review comment:
       @dajac Can you take a look at this pr again? Thanks.




-- 
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] wenbingshen commented on pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

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


   > @wenbingshen感谢您的更新代码。经过质量检查合格后,我将其合并。
   
   Thank you very much for your guidance and help. :)


-- 
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] chia7712 merged pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

Posted by GitBox <gi...@apache.org>.
chia7712 merged pull request #10558:
URL: https://github.com/apache/kafka/pull/10558


   


-- 
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] chia7712 merged pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

Posted by GitBox <gi...@apache.org>.
chia7712 merged pull request #10558:
URL: https://github.com/apache/kafka/pull/10558


   


-- 
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] wenbingshen commented on pull request #10558: KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand

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


   > @wenbingshen Thanks for the patch. Should we add a unit test to verify this or perhaps extend an existing one?
   
   @dajac Thank you for your comment. Later today I will see if we add a unit test or expand from the existing one. I prefer to add a unit test. :)


-- 
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