From 5013a41a514973e9612012413832e063ab062aef Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 14 Oct 2015 14:06:14 -0700 Subject: [PATCH] =?UTF-8?q?KAFKA-2650:=20Change=20ConfigCommand=20--delete?= =?UTF-8?q?d-config=20option=20to=20align=20wit=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …h TopicCommand Author: Grant Henke Reviewers: Gwen Shapira Closes #308 from granthenke/configcommand --- .../scala/kafka/admin/ConfigCommand.scala | 22 +++++++++---------- .../unit/kafka/admin/ConfigCommandTest.scala | 10 ++++----- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 17bbbcbd8f3..f0217de6c42 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -94,7 +94,7 @@ object ConfigCommand { } private[admin] def parseConfigsToBeAdded(opts: ConfigCommandOptions): Properties = { - val configsToBeAdded = opts.options.valuesOf(opts.addedConfig).map(_.split("""\s*=\s*""")) + val configsToBeAdded = opts.options.valuesOf(opts.addConfig).map(_.split("""\s*=\s*""")) require(configsToBeAdded.forall(config => config.length == 2), "Invalid entity config: all configs to be added must be in the format \"key=val\".") val props = new Properties @@ -103,8 +103,8 @@ object ConfigCommand { } private[admin] def parseConfigsToBeDeleted(opts: ConfigCommandOptions): Seq[String] = { - if (opts.options.has(opts.deletedConfig)) { - val configsToBeDeleted = opts.options.valuesOf(opts.deletedConfig).map(_.trim()) + if (opts.options.has(opts.deleteConfig)) { + val configsToBeDeleted = opts.options.valuesOf(opts.deleteConfig).map(_.trim()) val propsToBeDeleted = new Properties configsToBeDeleted.foreach(propsToBeDeleted.setProperty(_, "")) configsToBeDeleted @@ -130,20 +130,20 @@ object ConfigCommand { .ofType(classOf[String]) val nl = System.getProperty("line.separator") - val addedConfig = parser.accepts("added-config", "Key Value pairs configs to add 'k1=v1,k2=v2'. The following is a list of valid configurations: " + + val addConfig = parser.accepts("add-config", "Key Value pairs configs to add 'k1=v1,k2=v2'. The following is a list of valid configurations: " + "For entity_type '" + ConfigType.Topic + "': " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl + "For entity_type '" + ConfigType.Client + "' currently no configs are processed by the brokers") .withRequiredArg .ofType(classOf[String]) .withValuesSeparatedBy(',') - val deletedConfig = parser.accepts("deleted-config", "config keys to remove 'k1,k2'") + val deleteConfig = parser.accepts("delete-config", "config keys to remove 'k1,k2'") .withRequiredArg .ofType(classOf[String]) .withValuesSeparatedBy(',') val helpOpt = parser.accepts("help", "Print usage information.") val options = parser.parse(args : _*) - val allOpts: Set[OptionSpec[_]] = Set(alterOpt, describeOpt, entityType, entityName, addedConfig, deletedConfig, helpOpt) + val allOpts: Set[OptionSpec[_]] = Set(alterOpt, describeOpt, entityType, entityName, addConfig, deleteConfig, helpOpt) def checkArgs() { // should have exactly one action @@ -154,15 +154,15 @@ object ConfigCommand { // check required args CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt, entityType) CommandLineUtils.checkInvalidArgs(parser, options, alterOpt, Set(describeOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, describeOpt, Set(alterOpt, addedConfig, deletedConfig)) + CommandLineUtils.checkInvalidArgs(parser, options, describeOpt, Set(alterOpt, addConfig, deleteConfig)) if(options.has(alterOpt)) { if(! options.has(entityName)) throw new IllegalArgumentException("--entity-name must be specified with --alter") - val isAddedPresent: Boolean = options.has(addedConfig) - val isDeletedPresent: Boolean = options.has(deletedConfig) - if(! isAddedPresent && ! isDeletedPresent) - throw new IllegalArgumentException("At least one of --added-config or --deleted-config must be specified with --alter") + val isAddConfigPresent: Boolean = options.has(addConfig) + val isDeleteConfigPresent: Boolean = options.has(deleteConfig) + if(! isAddConfigPresent && ! isDeleteConfigPresent) + throw new IllegalArgumentException("At least one of --add-config or --delete-config must be specified with --alter") } val entityTypeVal = options.valueOf(entityType) if(! entityTypeVal.equals(ConfigType.Topic) && ! entityTypeVal.equals(ConfigType.Client)) { diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala index 0379064a360..6f7141a2c98 100644 --- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -37,7 +37,7 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { "--entity-name", "x", "--entity-type", "clients", "--alter", - "--added-config", "a=b,c=d")) + "--add-config", "a=b,c=d")) createOpts.checkArgs() // For alter and deleted config @@ -45,7 +45,7 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { "--entity-name", "x", "--entity-type", "clients", "--alter", - "--deleted-config", "a,b,c")) + "--delete-config", "a,b,c")) createOpts.checkArgs() // For alter and both added, deleted config @@ -53,8 +53,8 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { "--entity-name", "x", "--entity-type", "clients", "--alter", - "--added-config", "a=b,c=d", - "--deleted-config", "a")) + "--add-config", "a=b,c=d", + "--delete-config", "a")) createOpts.checkArgs() val addedProps = ConfigCommand.parseConfigsToBeAdded(createOpts) assertEquals(2, addedProps.size()) @@ -65,4 +65,4 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { assertEquals(1, deletedProps.size) assertEquals("a", deletedProps(0)) } -} \ No newline at end of file +}