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/01 09:24:26 UTC

[GitHub] [kafka] showuon opened a new pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

showuon opened a new pull request #10457:
URL: https://github.com/apache/kafka/pull/10457


   Remove `ZookeeperTopicService` and the test using zookeeper `TopicCommandWithZKClientTest`
   
   ### 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] showuon edited a comment on pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

Posted by GitBox <gi...@apache.org>.
showuon edited a comment on pull request #10457:
URL: https://github.com/apache/kafka/pull/10457#issuecomment-812456367


   @ijuma , thanks for the good suggestion. Yes, we don't need the `TopicService` trait now. Updated. Please help review it again. Thank you.


-- 
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 #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala
##########
@@ -98,15 +100,15 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
   @Test
   def testCreate(): Unit = {
     createAndWaitTopic(new TopicCommandOptions(
-      Array("--partitions", "2", "--replication-factor", "1", "--topic", testTopicName)))
+      brokerOptions ++ Array("--partitions", "2", "--replication-factor", "1", "--topic", testTopicName)))

Review comment:
       We didn't `checkArgs` before, so it's ok to not pass the `--bootstrap-server` option. Since we make it as required argument now, we need to pass the option explicitly.




-- 
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 #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   One more thing, did we check that there are no system tests using the flag we just removed?


-- 
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 pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   @ijuma , please help review 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] showuon commented on a change in pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -739,18 +563,15 @@ object TopicCommand extends Logging {
         CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete")
 
       // check required args
-      if (has(bootstrapServerOpt) == has(zkConnectOpt))
-        throw new IllegalArgumentException("Only one of --bootstrap-server or --zookeeper must be specified")
-
       if (!has(bootstrapServerOpt))
-        CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt)
-      if(has(describeOpt) && has(ifExistsOpt))
+        throw new IllegalArgumentException("--bootstrap-server must be specified")
+      if (has(describeOpt) && has(ifExistsOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
       if (!has(listOpt) && !has(describeOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
-      if (has(createOpt) && !has(replicaAssignmentOpt) && has(zkConnectOpt))
+      if (has(createOpt) && !has(replicaAssignmentOpt))

Review comment:
       Good catch! Added 2 tests: `testCreateWithAssignmentAndPartitionCount` and `testCreateWithAssignmentAndReplicationFactor` in `TopicCommandTest`. Thank you.




-- 
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 a change in pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -670,9 +505,9 @@ object TopicCommand extends Logging {
     private val reportUnavailablePartitionsOpt = parser.accepts("unavailable-partitions",
       "if set when describing topics, only show partitions whose leader is not available")
     private val reportUnderMinIsrPartitionsOpt = parser.accepts("under-min-isr-partitions",
-      "if set when describing topics, only show partitions whose isr count is less than the configured minimum. Not supported with the --zookeeper option.")
+      "if set when describing topics, only show partitions whose isr count is less than the configured minimum.")
     private val reportAtMinIsrPartitionsOpt = parser.accepts("at-min-isr-partitions",
-      "if set when describing topics, only show partitions whose isr count is equal to the configured minimum. Not supported with the --zookeeper option.")
+      "if set when describing topics, only show partitions whose isr count is equal to the configured minimum.")

Review comment:
       We have this line:
   
   ```
       // This is not currently used, but we keep it for compatibility
       parser.accepts("force", "Suppress console prompts")
   ```
   
   Was this deprecated also? Can we remove it?




-- 
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 #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala
##########
@@ -65,9 +65,10 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
   private val numPartitions = 1

Review comment:
       @ijuma , we already have a `TopicCommandTest` to do Unit test. This `TopicCommandWithAdminClientTest` is much like an integration test. How about rename to `TopicCommandIntegrationTest` just like `DeleteOffsetsConsumerGroupCommandIntegrationTest` in the same folder?




-- 
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 pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   @ijuma , Thanks for the comments. I've updated. Please take a look again. Thank you very much.


-- 
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 a change in pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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



##########
File path: core/src/test/scala/unit/kafka/admin/TopicCommandIntegrationTest.scala
##########
@@ -83,8 +85,9 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
     // create adminClient
     val props = new Properties()
     props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerList)
+    brokerOptions = Array("--bootstrap-server", brokerList)

Review comment:
       Why do we need this now?




-- 
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 #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -229,13 +205,21 @@ object TopicCommand extends Logging {
       Admin.create(commandConfig)
     }
 
-    def apply(commandConfig: Properties, bootstrapServer: Option[String]): AdminClientTopicService =
-      new AdminClientTopicService(createAdminClient(commandConfig, bootstrapServer))
+    def apply(commandConfig: Properties, bootstrapServer: Option[String]): TopicService =
+      new TopicService(createAdminClient(commandConfig, bootstrapServer))
   }
 
-  case class AdminClientTopicService private (adminClient: Admin) extends TopicService {
+  case class TopicService private (adminClient: Admin) extends AutoCloseable {

Review comment:
       Rename `AdminClientTopicService` into `TopicService`




-- 
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 #10457: KAFKA-12596: remove --zookeeper option from topic command

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



##########
File path: core/src/test/scala/unit/kafka/admin/TopicCommandIntegrationTest.scala
##########
@@ -83,8 +85,9 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
     // create adminClient
     val props = new Properties()
     props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerList)
+    brokerOptions = Array("--bootstrap-server", brokerList)

Review comment:
       Oops! Sorry, it should be removed! 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] ijuma edited a comment on pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

Posted by GitBox <gi...@apache.org>.
ijuma edited a comment on pull request #10457:
URL: https://github.com/apache/kafka/pull/10457#issuecomment-852149539


   After merging, I noticed we are missing a note in upgrade.html. We can fix that as part of #10471.


-- 
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 a change in pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -739,16 +573,14 @@ object TopicCommand extends Logging {
         CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete")
 
       // check required args
-      if (has(bootstrapServerOpt) == has(zkConnectOpt))
-        throw new IllegalArgumentException("Only one of --bootstrap-server or --zookeeper must be specified")
-
       if (!has(bootstrapServerOpt))
-        CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt)
+        throw new IllegalArgumentException("--bootstrap-server must be specified")
+
       if(has(describeOpt) && has(ifExistsOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
       if (!has(listOpt) && !has(describeOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
-      if (has(createOpt) && !has(replicaAssignmentOpt) && has(zkConnectOpt))
+      if (has(createOpt) && !has(replicaAssignmentOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, partitionsOpt, replicationFactorOpt)
       if (has(bootstrapServerOpt) && has(alterOpt)) {

Review comment:
       We don't need the `has(bootstrapServerOpt)` check.




-- 
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 #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   After merging, I noticed we are missing a note in upgrade.html. We can fix that as part of #10471,


-- 
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 pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   Will address the above 2 comments in #10471. Thank you.


-- 
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 #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala
##########
@@ -65,9 +65,10 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
   private val numPartitions = 1

Review comment:
       I see. Moved back now. 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] showuon commented on a change in pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -610,20 +441,18 @@ object TopicCommand extends Logging {
   }
 
   class TopicCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
-    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to. In case of providing this, a direct Zookeeper connection won't be required.")
+    private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to.")
       .withRequiredArg
       .describedAs("server to connect to")
       .ofType(classOf[String])
+      .required()

Review comment:
       Set as required argument.




-- 
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 pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   @ijuma  , could you help check 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] showuon commented on pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   @ijuma , could you please check this PR? Thank you.
   Failed tests are all un-related.
   ```
       Build / JDK 15 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateAndManyTopicsWithManyPartitions()
       Build / JDK 8 and Scala 2.12 / kafka.admin.ReassignPartitionsIntegrationTest.testLogDirReassignment()
   ```


-- 
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 a change in pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala
##########
@@ -65,9 +65,10 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
   private val numPartitions = 1

Review comment:
       We can rename this test to be `TopicCommandTest`.




-- 
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 pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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


   @ijuma @tombentley , could you help review this PR? 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] showuon commented on pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   @ijuma , I've updated the PR to check `--bootstrap-server` option manually, so that the help text can print without arguments. 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] ijuma commented on a change in pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -739,18 +563,15 @@ object TopicCommand extends Logging {
         CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete")
 
       // check required args
-      if (has(bootstrapServerOpt) == has(zkConnectOpt))
-        throw new IllegalArgumentException("Only one of --bootstrap-server or --zookeeper must be specified")
-
       if (!has(bootstrapServerOpt))
-        CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt)
-      if(has(describeOpt) && has(ifExistsOpt))
+        throw new IllegalArgumentException("--bootstrap-server must be specified")
+      if (has(describeOpt) && has(ifExistsOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
       if (!has(listOpt) && !has(describeOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
-      if (has(createOpt) && !has(replicaAssignmentOpt) && has(zkConnectOpt))
+      if (has(createOpt) && !has(replicaAssignmentOpt))

Review comment:
       Was it a bug that we only verified this when `zkConnectOpt` was set? If so, we should add a unit test for 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] showuon commented on a change in pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -670,9 +505,9 @@ object TopicCommand extends Logging {
     private val reportUnavailablePartitionsOpt = parser.accepts("unavailable-partitions",
       "if set when describing topics, only show partitions whose leader is not available")
     private val reportUnderMinIsrPartitionsOpt = parser.accepts("under-min-isr-partitions",
-      "if set when describing topics, only show partitions whose isr count is less than the configured minimum. Not supported with the --zookeeper option.")
+      "if set when describing topics, only show partitions whose isr count is less than the configured minimum.")
     private val reportAtMinIsrPartitionsOpt = parser.accepts("at-min-isr-partitions",
-      "if set when describing topics, only show partitions whose isr count is equal to the configured minimum. Not supported with the --zookeeper option.")
+      "if set when describing topics, only show partitions whose isr count is equal to the configured minimum.")

Review comment:
       I confirmed this issue is fixed in KIP-74 (KAFKA-2063). Yes I'll remove it!. Thank you.




-- 
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 #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala
##########
@@ -65,9 +65,10 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
   private val numPartitions = 1

Review comment:
       So, I rename the test to `TopicCommandIntegrationTest`, and move this test to integration test folder: `core/src/test/scala/integration/kafka/admin`. 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] showuon commented on pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   Failed tests are unrelated and flaky. Thanks.
   ```
       Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
       Build / JDK 11 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateListDeleteTopic()
       Build / JDK 15 and Scala 2.13 / org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTest.testReplication()
   ```


-- 
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 a change in pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala
##########
@@ -65,9 +65,10 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
   private val numPartitions = 1

Review comment:
       The name sounds good, but I think you don't have to move the test. We rely on tags instead of directory structure for distinguishing test types.




-- 
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 #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   Unrelated failures:
   
   > Build / JDK 8 and Scala 2.12 / testReplicationWithEmptyPartition() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest
   > 4s
   > Build / JDK 11 and Scala 2.13 / testCreateClusterAndCreateListDeleteTopic() – kafka.server.RaftClusterTest
   > 15s
   > Build / JDK 15 and Scala 2.13 / testReplication() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTest


-- 
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 a change in pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -229,13 +205,21 @@ object TopicCommand extends Logging {
       Admin.create(commandConfig)
     }
 
-    def apply(commandConfig: Properties, bootstrapServer: Option[String]): AdminClientTopicService =
-      new AdminClientTopicService(createAdminClient(commandConfig, bootstrapServer))
+    def apply(commandConfig: Properties, bootstrapServer: Option[String]): TopicService =
+      new TopicService(createAdminClient(commandConfig, bootstrapServer))
   }
 
-  case class AdminClientTopicService private (adminClient: Admin) extends TopicService {
+  case class TopicService private (adminClient: Admin) extends AutoCloseable {

Review comment:
       This should not be a case class.




-- 
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 #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -204,23 +196,7 @@ object TopicCommand extends Logging {
     }
   }
 
-  trait TopicService extends AutoCloseable {

Review comment:
       remove `TopicService` trait since we only have one service now.




-- 
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 a change in pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -739,16 +573,14 @@ object TopicCommand extends Logging {
         CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete")
 
       // check required args
-      if (has(bootstrapServerOpt) == has(zkConnectOpt))
-        throw new IllegalArgumentException("Only one of --bootstrap-server or --zookeeper must be specified")
-
       if (!has(bootstrapServerOpt))
-        CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt)
+        throw new IllegalArgumentException("--bootstrap-server must be specified")

Review comment:
       I think you can mark the argument as required to get this behavior automatically.




-- 
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 pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   failed tests are unrelated and flaky. Thanks.
   ```
       Build / JDK 15 and Scala 2.13 / org.apache.kafka.connect.integration.BlockingConnectorTest.testBlockInSinkTaskStart
       Build / JDK 15 and Scala 2.13 / org.apache.kafka.connect.integration.BlockingConnectorTest.testBlockInConnectorStop
       Build / JDK 11 and Scala 2.13 / org.apache.kafka.connect.integration.SourceConnectorsIntegrationTest.testTopicsAreCreatedWhenAutoCreateTopicsIsEnabledAtTheBroker
       Build / JDK 11 and Scala 2.13 / kafka.integration.MetricsDuringTopicCreationDeletionTest.testMetricsDuringTopicCreateDelete()
   ```


-- 
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 merged pull request #10457: KAFKA-12596: remove --zookeeper option from topic command

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


   


-- 
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 pull request #10457: KAFKA-12588: remove --zookeeper option from topic command

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


   @ijuma , thanks for the good suggestion. Please help review it again. Thank you.


-- 
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 #10457: KAFKA-12588: remove --zookeeper option from topic command

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



##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -739,16 +573,14 @@ object TopicCommand extends Logging {
         CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete")
 
       // check required args
-      if (has(bootstrapServerOpt) == has(zkConnectOpt))
-        throw new IllegalArgumentException("Only one of --bootstrap-server or --zookeeper must be specified")
-
       if (!has(bootstrapServerOpt))
-        CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt)
+        throw new IllegalArgumentException("--bootstrap-server must be specified")

Review comment:
       Good suggestion! Updated

##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -739,16 +573,14 @@ object TopicCommand extends Logging {
         CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete")
 
       // check required args
-      if (has(bootstrapServerOpt) == has(zkConnectOpt))
-        throw new IllegalArgumentException("Only one of --bootstrap-server or --zookeeper must be specified")
-
       if (!has(bootstrapServerOpt))
-        CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt)
+        throw new IllegalArgumentException("--bootstrap-server must be specified")
+
       if(has(describeOpt) && has(ifExistsOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
       if (!has(listOpt) && !has(describeOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
-      if (has(createOpt) && !has(replicaAssignmentOpt) && has(zkConnectOpt))
+      if (has(createOpt) && !has(replicaAssignmentOpt))
         CommandLineUtils.checkRequiredArgs(parser, options, partitionsOpt, replicationFactorOpt)
       if (has(bootstrapServerOpt) && has(alterOpt)) {

Review comment:
       You're right.




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