hachikuji commented on code in PR #12469:
URL: https://github.com/apache/kafka/pull/12469#discussion_r941548893


##########
core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala:
##########
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.admin
+
+import kafka.tools.TerseFailure
+import kafka.utils.Exit
+import net.sourceforge.argparse4j.ArgumentParsers
+import net.sourceforge.argparse4j.impl.Arguments.fileType
+import net.sourceforge.argparse4j.inf.Subparsers
+import org.apache.kafka.clients._
+import org.apache.kafka.clients.admin.{Admin, QuorumInfo}
+import org.apache.kafka.common.utils.Utils
+
+import java.io.File
+import java.util.Properties
+import scala.jdk.CollectionConverters._
+
+/**
+ * A tool for describing quorum status
+ */
+object MetadataQuorumCommand {
+
+  def main(args: Array[String]): Unit = {
+    val res = mainNoExit(args)
+    Exit.exit(res)
+  }
+
+  def mainNoExit(args: Array[String]): Int = {
+    val parser = ArgumentParsers.newArgumentParser("kafka-metadata-quorum")
+      .defaultHelp(true)
+      .description("This tool describes kraft metadata quorum status.")
+    parser.addArgument("--bootstrap-server")
+      .help("A comma-separated list of host:port pairs to use for establishing 
the connection to the Kafka cluster.")
+      .required(true)
+
+    parser.addArgument("--command-config")
+      .`type`(fileType())
+      .help("Property file containing configs to be passed to Admin Client.")
+    val subparsers = parser.addSubparsers().dest("command")
+    addDescribeParser(subparsers)
+
+    try {
+      val namespace = parser.parseArgsOrFail(args)
+      val command = namespace.getString("command")
+
+      val commandConfig = namespace.get[File]("command_config")
+      val props = if (commandConfig != null) {
+        if (!commandConfig.exists()) {
+          throw new TerseFailure(s"Properties file ${commandConfig.getPath} 
does not exists!")
+        }
+        Utils.loadProps(commandConfig.getPath)
+      } else {
+        new Properties()
+      }
+      props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, 
namespace.getString("bootstrap_server"))
+      val admin = Admin.create(props)
+
+      if (command == "describe") {
+        handleDescribe(admin)
+      } else {
+        // currently we only support describe
+      }
+      admin.close()
+      0
+    } catch {
+      case e: TerseFailure =>
+        Console.err.println(e.getMessage)
+        1
+    }
+  }
+
+  def addDescribeParser(subparsers: Subparsers): Unit = {
+    subparsers.addParser("describe")
+      .help("Describe the metadata quorum info")
+  }
+
+  def handleDescribe(admin: Admin): Unit = {

Review Comment:
   The output seems a bit different from what was documented in KIP-595 (and 
KIP-836):
   - 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum#KIP595:ARaftProtocolfortheMetadataQuorum-ToolingSupport.
 
   - 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-836:+Addition+of+Information+in+DescribeQuorumResponse+about+Voter+Lag
   
   Any reason to change it?



##########
core/src/main/scala/kafka/admin/MetadataQuorumCommand.scala:
##########
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.admin
+
+import kafka.tools.TerseFailure
+import kafka.utils.Exit
+import net.sourceforge.argparse4j.ArgumentParsers
+import net.sourceforge.argparse4j.impl.Arguments.fileType
+import net.sourceforge.argparse4j.inf.Subparsers
+import org.apache.kafka.clients._
+import org.apache.kafka.clients.admin.{Admin, QuorumInfo}
+import org.apache.kafka.common.utils.Utils
+
+import java.io.File
+import java.util.Properties
+import scala.jdk.CollectionConverters._
+
+/**
+ * A tool for describing quorum status
+ */
+object MetadataQuorumCommand {
+
+  def main(args: Array[String]): Unit = {
+    val res = mainNoExit(args)
+    Exit.exit(res)
+  }
+
+  def mainNoExit(args: Array[String]): Int = {
+    val parser = ArgumentParsers.newArgumentParser("kafka-metadata-quorum")
+      .defaultHelp(true)
+      .description("This tool describes kraft metadata quorum status.")
+    parser.addArgument("--bootstrap-server")
+      .help("A comma-separated list of host:port pairs to use for establishing 
the connection to the Kafka cluster.")
+      .required(true)
+
+    parser.addArgument("--command-config")
+      .`type`(fileType())
+      .help("Property file containing configs to be passed to Admin Client.")
+    val subparsers = parser.addSubparsers().dest("command")
+    addDescribeParser(subparsers)
+
+    try {
+      val namespace = parser.parseArgsOrFail(args)
+      val command = namespace.getString("command")
+
+      val commandConfig = namespace.get[File]("command_config")
+      val props = if (commandConfig != null) {
+        if (!commandConfig.exists()) {
+          throw new TerseFailure(s"Properties file ${commandConfig.getPath} 
does not exists!")
+        }
+        Utils.loadProps(commandConfig.getPath)
+      } else {
+        new Properties()
+      }
+      props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, 
namespace.getString("bootstrap_server"))
+      val admin = Admin.create(props)
+
+      if (command == "describe") {
+        handleDescribe(admin)
+      } else {
+        // currently we only support describe

Review Comment:
   Should we throw an exception?



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to