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/06/05 20:52:59 UTC

[GitHub] [kafka] dongjinleekr opened a new pull request #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

dongjinleekr opened a new pull request #10827:
URL: https://github.com/apache/kafka/pull/10827


   ### 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] tombentley commented on a change in pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")

Review comment:
       See later comment as for bootstrap-server.

##########
File path: core/src/test/scala/kafka/tools/ReplicaVerificationToolTest.scala
##########
@@ -17,14 +17,40 @@
 
 package kafka.tools
 
+import kafka.tools.ReplicaVerificationTool.ReplicaVerificationToolOptions
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.message.FetchResponseData
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
 import org.junit.jupiter.api.Test
-import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
+import kafka.utils.Exit
 
 class ReplicaVerificationToolTest {
 
+  @Test
+  def testExitWithoutBootstrapServers(): Unit = {
+    Exit.setExitProcedure {
+      (exitCode: Int, _: Option[String]) =>
+        assertEquals(1, exitCode)
+        throw new RuntimeException
+    }
+
+    try assertThrows(classOf[RuntimeException], () => new ReplicaVerificationToolOptions(Array("--fetch-size", "1024")))
+    finally Exit.resetExitProcedure()
+  }
+
+  @Test
+  def testConfigOptWithBootstrapServers(): Unit = {
+    val opts1 = new ReplicaVerificationToolOptions(Array("--bootstrap-server", "localhost:9092"))
+    assertEquals("localhost:9092", opts1.bootstrapServer)
+
+    val opts2 = new ReplicaVerificationToolOptions(Array("--broker-list", "127.0.0.1:9092"))
+    assertEquals("127.0.0.1:9092", opts2.bootstrapServer)
+
+    val opts3 = new ReplicaVerificationToolOptions(Array("--broker-list", "127.0.0.1:9092", "--bootstrap-server", "localhost:9092"))
+    assertEquals("localhost:9092", opts3.bootstrapServer)
+  }

Review comment:
       We should have an assertion with multiple `--bootstrap-servers` given.

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
+      .withRequiredArg
+      .describedAs("bytes")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
+    private val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
+      .withRequiredArg
+      .describedAs("ms")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1000)
     val topicWhiteListOpt = parser.accepts("topic-white-list", "White list of topics to verify replica consistency. Defaults to all topics.")

Review comment:
       If we're changing this class we ought to deprecate this option and replace it with one with a less contentious name.

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")

Review comment:
       `TopicCommandOtions` and `ConfigCommandOptions` describes this as `.describedAs("server to connect to")`, so I guess we should do the same here.




-- 
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] dongjinleekr commented on pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   @tombentley Could you kindly have a look when you are free? :bowing_man:


-- 
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] dongjinleekr commented on pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   @kkonstantine It would be really great for me if this PR is included in 3.0, but it still needs one more binding +1. (It is also why it is prefixed with '[WIP]'.) Could you have a look at [the voting thread](https://lists.apache.org/thread.html/rebd427d5fd34acf5b378d7a904af2c804e7460b32d34ddbb3368776c%40%3Cdev.kafka.apache.org%3E) and cast a vote? If so, it may be included in the next release.
   
   Besides, I think we should merge [KAFKA-10589](https://github.com/apache/kafka/pull/11007) before this one; you know, [KIP-629](https://cwiki.apache.org/confluence/display/KAFKA/KIP-629:+Use+racially+neutral+terms+in+our+codebase) is already passed.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] dongjinleekr commented on pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   Hi @tombentley
   
   Here is the update. I added additional tests (`ReplicaVerificationToolTest#{testExitWithMultipleBrokerLists, testExitWithMultipleBootstrapServers`), Add descriptions to exising test, with changing parameter descriptions.


-- 
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] dongjinleekr commented on a change in pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
+      .withRequiredArg
+      .describedAs("bytes")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
+    private val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
+      .withRequiredArg
+      .describedAs("ms")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1000)
     val topicWhiteListOpt = parser.accepts("topic-white-list", "White list of topics to verify replica consistency. Defaults to all topics.")

Review comment:
       Well, `fetchSizeOpt` and `maxWaitMsOpt` are precisely the same as the previous ones. Which change do you mean? :thinking:




-- 
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] dongjinleekr commented on a change in pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/test/scala/kafka/tools/ReplicaVerificationToolTest.scala
##########
@@ -17,14 +17,40 @@
 
 package kafka.tools
 
+import kafka.tools.ReplicaVerificationTool.ReplicaVerificationToolOptions
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.message.FetchResponseData
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
 import org.junit.jupiter.api.Test
-import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
+import kafka.utils.Exit
 
 class ReplicaVerificationToolTest {
 
+  @Test
+  def testExitWithoutBootstrapServers(): Unit = {
+    Exit.setExitProcedure {
+      (exitCode: Int, _: Option[String]) =>
+        assertEquals(1, exitCode)
+        throw new RuntimeException
+    }
+
+    try assertThrows(classOf[RuntimeException], () => new ReplicaVerificationToolOptions(Array("--fetch-size", "1024")))
+    finally Exit.resetExitProcedure()
+  }
+
+  @Test
+  def testConfigOptWithBootstrapServers(): Unit = {
+    val opts1 = new ReplicaVerificationToolOptions(Array("--bootstrap-server", "localhost:9092"))
+    assertEquals("localhost:9092", opts1.bootstrapServer)
+
+    val opts2 = new ReplicaVerificationToolOptions(Array("--broker-list", "127.0.0.1:9092"))
+    assertEquals("127.0.0.1:9092", opts2.bootstrapServer)
+
+    val opts3 = new ReplicaVerificationToolOptions(Array("--broker-list", "127.0.0.1:9092", "--bootstrap-server", "localhost:9092"))
+    assertEquals("localhost:9092", opts3.bootstrapServer)
+  }

Review comment:
       Great. :+1: 




-- 
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] dongjinleekr commented on a change in pull request #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -116,7 +121,13 @@ object ReplicaVerificationTool extends Logging {
     if (options.has(versionOpt)) {
       CommandLineUtils.printVersionAndDie()
     }
-    CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt)
+
+    val effectiveBrokerListOpt = if (options.has(bootstrapServerOpt))
+      bootstrapServerOpt
+    else
+      brokerListOpt

Review comment:
       Yes. But `bootstrapServerOpt` has the preference. (`brokerListOpt` is deprecated, but not removed yet.)




-- 
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] tombentley commented on a change in pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
+      .withRequiredArg
+      .describedAs("bytes")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
+    private val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
+      .withRequiredArg
+      .describedAs("ms")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1000)
     val topicWhiteListOpt = parser.accepts("topic-white-list", "White list of topics to verify replica consistency. Defaults to all topics.")

Review comment:
       whitelist is a term we should avoid (see [KIP-629](https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase) and [KAFKA-10201](https://issues.apache.org/jira/browse/KAFKA-10201)).




-- 
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] tombentley commented on a change in pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")

Review comment:
       See later comment as for bootstrap-server.

##########
File path: core/src/test/scala/kafka/tools/ReplicaVerificationToolTest.scala
##########
@@ -17,14 +17,40 @@
 
 package kafka.tools
 
+import kafka.tools.ReplicaVerificationTool.ReplicaVerificationToolOptions
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.message.FetchResponseData
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
 import org.junit.jupiter.api.Test
-import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
+import kafka.utils.Exit
 
 class ReplicaVerificationToolTest {
 
+  @Test
+  def testExitWithoutBootstrapServers(): Unit = {
+    Exit.setExitProcedure {
+      (exitCode: Int, _: Option[String]) =>
+        assertEquals(1, exitCode)
+        throw new RuntimeException
+    }
+
+    try assertThrows(classOf[RuntimeException], () => new ReplicaVerificationToolOptions(Array("--fetch-size", "1024")))
+    finally Exit.resetExitProcedure()
+  }
+
+  @Test
+  def testConfigOptWithBootstrapServers(): Unit = {
+    val opts1 = new ReplicaVerificationToolOptions(Array("--bootstrap-server", "localhost:9092"))
+    assertEquals("localhost:9092", opts1.bootstrapServer)
+
+    val opts2 = new ReplicaVerificationToolOptions(Array("--broker-list", "127.0.0.1:9092"))
+    assertEquals("127.0.0.1:9092", opts2.bootstrapServer)
+
+    val opts3 = new ReplicaVerificationToolOptions(Array("--broker-list", "127.0.0.1:9092", "--bootstrap-server", "localhost:9092"))
+    assertEquals("localhost:9092", opts3.bootstrapServer)
+  }

Review comment:
       We should have an assertion with multiple `--bootstrap-servers` given.

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
+      .withRequiredArg
+      .describedAs("bytes")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
+    private val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
+      .withRequiredArg
+      .describedAs("ms")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1000)
     val topicWhiteListOpt = parser.accepts("topic-white-list", "White list of topics to verify replica consistency. Defaults to all topics.")

Review comment:
       If we're changing this class we ought to deprecate this option and replace it with one with a less contentious name.

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")

Review comment:
       `TopicCommandOtions` and `ConfigCommandOptions` describes this as `.describedAs("server to connect to")`, so I guess we should do the same here.

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
+      .withRequiredArg
+      .describedAs("bytes")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
+    private val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
+      .withRequiredArg
+      .describedAs("ms")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1000)
     val topicWhiteListOpt = parser.accepts("topic-white-list", "White list of topics to verify replica consistency. Defaults to all topics.")

Review comment:
       whitelist is a term we should avoid (see [KIP-629](https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase) and [KAFKA-10201](https://issues.apache.org/jira/browse/KAFKA-10201)).




-- 
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] dongjinleekr commented on a change in pull request #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -116,7 +121,13 @@ object ReplicaVerificationTool extends Logging {
     if (options.has(versionOpt)) {
       CommandLineUtils.printVersionAndDie()
     }
-    CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt)
+
+    val effectiveBrokerListOpt = if (options.has(bootstrapServerOpt))
+      bootstrapServerOpt
+    else
+      brokerListOpt

Review comment:
       Yes. But `bootstrapServerOpt` has the preference.




-- 
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] dongjinleekr commented on pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   Hi @tombentley,
   
   Sorry for missing your comment on [KIP-629](https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase?src=contextnavchildmode). Reviewing [KAFKA-10201](https://issues.apache.org/jira/browse/KAFKA-10201), I found that it is already under progress with [KAFKA-10589](https://issues.apache.org/jira/browse/KAFKA-10589). So, it would be better to address this issue in [this PR](https://github.com/apache/kafka/pull/9404).
   
   I also rebased onto the latest trunk. Thank you for reviewing! :bowing_man:


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] dongjinleekr commented on a change in pull request #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -116,7 +121,13 @@ object ReplicaVerificationTool extends Logging {
     if (options.has(versionOpt)) {
       CommandLineUtils.printVersionAndDie()
     }
-    CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt)
+
+    val effectiveBrokerListOpt = if (options.has(bootstrapServerOpt))
+      bootstrapServerOpt
+    else
+      brokerListOpt
+
+    CommandLineUtils.checkRequiredArgs(parser, options, effectiveBrokerListOpt)
 
     val regex = options.valueOf(topicWhiteListOpt)
     val topicWhiteListFiler = new IncludeList(regex)

Review comment:
       Fixed.




-- 
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] dongjinleekr commented on a change in pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/test/scala/kafka/tools/ReplicaVerificationToolTest.scala
##########
@@ -17,14 +17,40 @@
 
 package kafka.tools
 
+import kafka.tools.ReplicaVerificationTool.ReplicaVerificationToolOptions
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.message.FetchResponseData
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
 import org.junit.jupiter.api.Test
-import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
+import kafka.utils.Exit
 
 class ReplicaVerificationToolTest {
 
+  @Test
+  def testExitWithoutBootstrapServers(): Unit = {
+    Exit.setExitProcedure {
+      (exitCode: Int, _: Option[String]) =>
+        assertEquals(1, exitCode)
+        throw new RuntimeException
+    }
+
+    try assertThrows(classOf[RuntimeException], () => new ReplicaVerificationToolOptions(Array("--fetch-size", "1024")))
+    finally Exit.resetExitProcedure()
+  }
+
+  @Test
+  def testConfigOptWithBootstrapServers(): Unit = {
+    val opts1 = new ReplicaVerificationToolOptions(Array("--bootstrap-server", "localhost:9092"))
+    assertEquals("localhost:9092", opts1.bootstrapServer)
+
+    val opts2 = new ReplicaVerificationToolOptions(Array("--broker-list", "127.0.0.1:9092"))
+    assertEquals("127.0.0.1:9092", opts2.bootstrapServer)
+
+    val opts3 = new ReplicaVerificationToolOptions(Array("--broker-list", "127.0.0.1:9092", "--bootstrap-server", "localhost:9092"))
+    assertEquals("localhost:9092", opts3.bootstrapServer)
+  }

Review comment:
       Great. :+1: 

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")

Review comment:
       `ConsoleConsumer`, `AclCommand`, `ConfigCommand` also do. :+1: 

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
+      .withRequiredArg
+      .describedAs("bytes")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
+    private val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
+      .withRequiredArg
+      .describedAs("ms")
+      .ofType(classOf[java.lang.Integer])
+      .defaultsTo(1000)
     val topicWhiteListOpt = parser.accepts("topic-white-list", "White list of topics to verify replica consistency. Defaults to all topics.")

Review comment:
       Well, `fetchSizeOpt` and `maxWaitMsOpt` are precisely the same as the previous ones. Which change do you mean? :thinking:




-- 
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] guozhangwang commented on pull request #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   Looks good to me. Will wait for the unit test to be added.


-- 
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] dongjinleekr commented on a change in pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -73,53 +74,61 @@ object ReplicaVerificationTool extends Logging {
     ReplicaVerificationTool.dateFormat.format(new Date(Time.SYSTEM.milliseconds))
   }
 
-  def main(args: Array[String]): Unit = {
-    val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
-                         .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
-                         .ofType(classOf[String])
-    val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.")
-                         .withRequiredArg
-                         .describedAs("bytes")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES)
-    val maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of time each fetch request waits.")
-                         .withRequiredArg
-                         .describedAs("ms")
-                         .ofType(classOf[java.lang.Integer])
-                         .defaultsTo(1000)
+  // Non-private for testing
+  sealed class ReplicaVerificationToolOptions(args: Array[String]) extends CommandDefaultOptions(args) {
+    private val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+      .ofType(classOf[String])
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")
+      .requiredUnless("broker-list")
+      .withRequiredArg
+      .describedAs("HOST1:PORT1,...,HOST3:PORT3")

Review comment:
       `ConsoleConsumer`, `AclCommand`, `ConfigCommand` also do. :+1: 




-- 
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] showuon commented on a change in pull request #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -116,7 +121,13 @@ object ReplicaVerificationTool extends Logging {
     if (options.has(versionOpt)) {
       CommandLineUtils.printVersionAndDie()
     }
-    CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt)
+
+    val effectiveBrokerListOpt = if (options.has(bootstrapServerOpt))
+      bootstrapServerOpt
+    else
+      brokerListOpt

Review comment:
       Is it possible that user provided both `bootstrapServerOpt` and `brokerListOpt` at the same time?

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -116,7 +121,13 @@ object ReplicaVerificationTool extends Logging {
     if (options.has(versionOpt)) {
       CommandLineUtils.printVersionAndDie()
     }
-    CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt)
+
+    val effectiveBrokerListOpt = if (options.has(bootstrapServerOpt))
+      bootstrapServerOpt
+    else
+      brokerListOpt
+
+    CommandLineUtils.checkRequiredArgs(parser, options, effectiveBrokerListOpt)
 
     val regex = options.valueOf(topicWhiteListOpt)
     val topicWhiteListFiler = new IncludeList(regex)

Review comment:
       typo: `topicWhiteListFiler` ->‵topicWhiteListFilter` 

##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -75,9 +75,14 @@ object ReplicaVerificationTool extends Logging {
 
   def main(args: Array[String]): Unit = {
     val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
+    val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
                          .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
+                         .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+                         .ofType(classOf[String])
+    val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")

Review comment:
       add tests +1




-- 
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 #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -75,9 +75,14 @@ object ReplicaVerificationTool extends Logging {
 
   def main(args: Array[String]): Unit = {
     val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
+    val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
                          .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
+                         .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+                         .ofType(classOf[String])
+    val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")

Review comment:
       Do we need to add a unit test for this new option?




-- 
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] dongjinleekr commented on a change in pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -116,7 +121,13 @@ object ReplicaVerificationTool extends Logging {
     if (options.has(versionOpt)) {
       CommandLineUtils.printVersionAndDie()
     }
-    CommandLineUtils.checkRequiredArgs(parser, options, brokerListOpt)
+
+    val effectiveBrokerListOpt = if (options.has(bootstrapServerOpt))
+      bootstrapServerOpt
+    else
+      brokerListOpt

Review comment:
       Fixed. See `ReplicaVerificationToolOptions#effectiveBrokerListOpt`.




-- 
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] kkonstantine commented on pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   @dongjinleekr @tombentley if we intend to include this fix to 3.0 we'll need to merge asap. Please let me know so I can update the status of https://issues.apache.org/jira/browse/KAFKA-12899 accordingly 


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] dongjinleekr commented on pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   Here it is. With rebasing onto the lastest trunk, I added unit tests that verify the command line parsing.
   
   Also, for this KIP is not passed yet (+2 bindings), I prefiexed the title with '[WIP]'.


-- 
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] dongjinleekr commented on pull request #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   @guozhangwang Yes, I am currently working on it. And could you kindly have a look at [the voting thread](https://lists.apache.org/thread.html/rebd427d5fd34acf5b378d7a904af2c804e7460b32d34ddbb3368776c%40%3Cdev.kafka.apache.org%3E)? As of present, +2 bindings with +2 non-bindings. One more binding is required to be passed.


-- 
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] dongjinleekr commented on a change in pull request #10827: KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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



##########
File path: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
##########
@@ -75,9 +75,14 @@ object ReplicaVerificationTool extends Logging {
 
   def main(args: Array[String]): Unit = {
     val parser = new OptionParser(false)
-    val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
+    val brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The list of hostname and port of the server to connect to.")
                          .withRequiredArg
-                         .describedAs("hostname:port,...,hostname:port")
+                         .describedAs("HOST1:PORT1,...,HOST3:PORT3")
+                         .ofType(classOf[String])
+    val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED. The list of hostname and port of the server to connect to.")

Review comment:
       I will. Stay tuned! :smile: 




-- 
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] dongjinleekr commented on pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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


   Rebased onto the latest trunk.


-- 
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] dongjinleekr commented on pull request #10827: [WIP] KAFKA-12899: Support --bootstrap-server in ReplicaVerificationTool

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






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