You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jg...@apache.org on 2018/03/15 17:49:52 UTC

[kafka] branch 1.0 updated: KAFKA-6656; Config tool should return non-zero status code on failure (#4711)

This is an automated email from the ASF dual-hosted git repository.

jgus pushed a commit to branch 1.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/1.0 by this push:
     new ca83677  KAFKA-6656; Config tool should return non-zero status code on failure (#4711)
ca83677 is described below

commit ca836774c7bfa1e17f4ed0dd8f6780db3dd74c64
Author: Jason Gustafson <ja...@confluent.io>
AuthorDate: Thu Mar 15 10:33:29 2018 -0700

    KAFKA-6656; Config tool should return non-zero status code on failure (#4711)
    
    Prior to this patch, we caught some exceptions when executing the command, which meant that it would return with status code zero. This patch fixes this and makes the expected exit behavior explicit. Test cases have been added to verify the change.
    
    Reviewers: Ismael Juma <is...@juma.me.uk>
---
 .../src/main/scala/kafka/admin/ConfigCommand.scala | 47 +++++++++++++---------
 .../scala/unit/kafka/admin/ConfigCommandTest.scala | 40 +++++++++++++++++-
 2 files changed, 65 insertions(+), 22 deletions(-)

diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala
index febf40f..c0cdff2 100644
--- a/core/src/main/scala/kafka/admin/ConfigCommand.scala
+++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala
@@ -24,7 +24,7 @@ import kafka.common.Config
 import kafka.common.InvalidConfigException
 import kafka.log.LogConfig
 import kafka.server.{ConfigEntityName, ConfigType, DynamicConfig}
-import kafka.utils.{CommandLineUtils, ZkUtils}
+import kafka.utils.{CommandLineUtils, Exit, ZkUtils}
 import kafka.utils.Implicits._
 import org.apache.kafka.common.security.JaasUtils
 import org.apache.kafka.common.security.scram._
@@ -53,30 +53,37 @@ object ConfigCommand extends Config {
   val DefaultScramIterations = 4096
 
   def main(args: Array[String]): Unit = {
+    try {
+      val opts = new ConfigCommandOptions(args)
 
-    val opts = new ConfigCommandOptions(args)
-
-    if(args.length == 0)
-      CommandLineUtils.printUsageAndDie(opts.parser, "Add/Remove entity config for a topic, client, user or broker")
+      if (args.length == 0)
+        CommandLineUtils.printUsageAndDie(opts.parser, "Add/Remove entity config for a topic, client, user or broker")
 
-    opts.checkArgs()
+      opts.checkArgs()
 
-    val zkUtils = ZkUtils(opts.options.valueOf(opts.zkConnectOpt),
-                          30000,
-                          30000,
-                          JaasUtils.isZkSecurityEnabled())
+      val zkUtils = ZkUtils(opts.options.valueOf(opts.zkConnectOpt),
+        30000,
+        30000,
+        JaasUtils.isZkSecurityEnabled())
 
-    try {
-      if (opts.options.has(opts.alterOpt))
-        alterConfig(zkUtils, opts)
-      else if (opts.options.has(opts.describeOpt))
-        describeConfig(zkUtils, opts)
+      try {
+        if (opts.options.has(opts.alterOpt))
+          alterConfig(zkUtils, opts)
+        else if (opts.options.has(opts.describeOpt))
+          describeConfig(zkUtils, opts)
+      } finally {
+        zkUtils.close()
+      }
     } catch {
-      case e: Throwable =>
-        println("Error while executing config command " + e.getMessage)
-        println(Utils.stackTrace(e))
-    } finally {
-      zkUtils.close()
+      case e @ (_: IllegalArgumentException | _: InvalidConfigException | _: OptionException) =>
+        logger.debug(s"Failed config command with args $args", e)
+        System.err.println(e.getMessage)
+        Exit.exit(1)
+
+      case t: Throwable =>
+        System.err.println(s"Error while executing config command with args $args")
+        t.printStackTrace(System.err)
+        Exit.exit(1)
     }
   }
 
diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
index 87ce46e..02591bd 100644
--- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala
@@ -21,8 +21,9 @@ import java.util.Properties
 import kafka.admin.ConfigCommand.ConfigCommandOptions
 import kafka.common.InvalidConfigException
 import kafka.server.ConfigEntityName
-import kafka.utils.{Logging, ZkUtils}
+import kafka.utils.ZkUtils
 import kafka.zk.ZooKeeperTestHarness
+import kafka.utils.{Exit, Logging}
 import org.apache.kafka.common.security.scram.ScramCredentialUtils
 import org.apache.kafka.common.utils.Sanitizer
 import org.easymock.EasyMock
@@ -33,6 +34,40 @@ import scala.collection.mutable
 import scala.collection.JavaConverters._
 
 class ConfigCommandTest extends ZooKeeperTestHarness with Logging {
+
+  @Test
+  def shouldExitWithNonZeroStatusOnArgError(): Unit = {
+    assertNonZeroStatusExit(Array("--blah"))
+  }
+
+  @Test
+  def shouldExitWithNonZeroStatusOnZkCommandError(): Unit = {
+    assertNonZeroStatusExit(Array(
+      "--zookeeper", zkConnect,
+      "--entity-name", "my-topic",
+      "--entity-type", "topics",
+      "--alter",
+      "--add-config", "message.max.size=100000"))
+  }
+
+  private def assertNonZeroStatusExit(args: Array[String]): Unit = {
+    var exitStatus: Option[Int] = None
+    Exit.setExitProcedure { (status, _) =>
+      exitStatus = Some(status)
+      throw new RuntimeException
+    }
+
+    try {
+      ConfigCommand.main(args)
+    } catch {
+      case e: RuntimeException =>
+    } finally {
+      Exit.resetExitProcedure()
+    }
+
+    assertEquals(Some(1), exitStatus)
+  }
+
   @Test
   def shouldParseArgumentsForClientsEntityType() {
     testArgumentParse("clients")
@@ -253,7 +288,7 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging {
         "--delete-config", mechanism))
 
     val credentials = mutable.Map[String, Properties]()
-    case class CredentialChange(val user: String, val mechanisms: Set[String], val iterations: Int) extends TestAdminUtils {
+    case class CredentialChange(user: String, mechanisms: Set[String], iterations: Int) extends TestAdminUtils {
       override def fetchEntityConfig(zkUtils: ZkUtils, entityType: String, entityName: String): Properties = {
         credentials.getOrElse(entityName, new Properties())
       }
@@ -456,4 +491,5 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging {
         Map("users" -> Seq("<default>", sanitizedPrincipal)) ++ defaultUserMap ++ userMap,
         Seq("<default>/clients/client-3", sanitizedPrincipal + "/clients/client-2"))
   }
+
 }

-- 
To stop receiving notification emails like this one, please contact
jgus@apache.org.