Repository: kafka Updated Branches: refs/heads/trunk 0df910c03 -> d4c4bcf01
MINOR: Make ConfigDef safer by not using empty string for NO_DEFAULT_VALUE. Author: Ewen Cheslack-Postava <[email protected]> Reviewers: Damian Guy <[email protected]>, Ismael Juma <[email protected]> Closes #2660 from ewencp/minor-make-configdef-safer Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d4c4bcf0 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d4c4bcf0 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d4c4bcf0 Branch: refs/heads/trunk Commit: d4c4bcf017a1b770dca71868b97d37ec7327ea00 Parents: 0df910c Author: Ewen Cheslack-Postava <[email protected]> Authored: Wed Apr 5 11:34:32 2017 +0100 Committer: Ismael Juma <[email protected]> Committed: Wed Apr 5 11:35:04 2017 +0100 ---------------------------------------------------------------------- .../java/org/apache/kafka/common/config/ConfigDef.java | 13 ++++++------- .../org/apache/kafka/common/config/ConfigDefTest.java | 6 ++++++ .../apache/kafka/connect/runtime/AbstractHerder.java | 4 ++-- .../rest/resources/ConnectorPluginsResourceTest.java | 6 +++--- 4 files changed, 17 insertions(+), 12 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kafka/blob/d4c4bcf0/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java ---------------------------------------------------------------------- diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index c4eac78..aac5b53 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -73,10 +73,9 @@ import java.util.Set; */ public class ConfigDef { /** - * A unique Java object which represents the lack of a default value.<p> - * The 'new' here is intentional. + * A unique Java object which represents the lack of a default value. */ - public static final Object NO_DEFAULT_VALUE = new String(""); + public static final Object NO_DEFAULT_VALUE = new Object(); private final Map<String, ConfigKey> configKeys; private final List<String> groups; @@ -457,7 +456,7 @@ public class ConfigDef { if (isSet) { parsedValue = parseType(key.name, value, key.type); // props map doesn't contain setting, the key is required because no default value specified - its an error - } else if (key.defaultValue == NO_DEFAULT_VALUE) { + } else if (NO_DEFAULT_VALUE.equals(key.defaultValue)) { throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value."); } else { // otherwise assign setting its default value @@ -559,7 +558,7 @@ public class ConfigDef { } catch (ConfigException e) { config.addErrorMessage(e.getMessage()); } - } else if (key.defaultValue == NO_DEFAULT_VALUE) { + } else if (NO_DEFAULT_VALUE.equals(key.defaultValue)) { config.addErrorMessage("Missing required configuration \"" + key.name + "\" which has no default value."); } else { value = key.defaultValue; @@ -922,7 +921,7 @@ public class ConfigDef { boolean internalConfig) { this.name = name; this.type = type; - this.defaultValue = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type); + this.defaultValue = NO_DEFAULT_VALUE.equals(defaultValue) ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type); this.validator = validator; this.importance = importance; if (this.validator != null && hasDefault()) @@ -938,7 +937,7 @@ public class ConfigDef { } public boolean hasDefault() { - return this.defaultValue != NO_DEFAULT_VALUE; + return !NO_DEFAULT_VALUE.equals(this.defaultValue); } } http://git-wip-us.apache.org/repos/asf/kafka/blob/d4c4bcf0/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java ---------------------------------------------------------------------- diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 11e5803..2113ce9 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -96,6 +96,12 @@ public class ConfigDefTest { new ConfigDef().define("a", Type.INT, Importance.HIGH, "docs").parse(new HashMap<String, Object>()); } + @Test + public void testParsingEmptyDefaultValueForStringFieldShouldSucceed() { + new ConfigDef().define("a", Type.STRING, "", ConfigDef.Importance.HIGH, "docs") + .parse(new HashMap<String, Object>()); + } + @Test(expected = ConfigException.class) public void testDefinedTwice() { new ConfigDef().define("a", Type.STRING, Importance.HIGH, "docs").define("a", Type.INT, Importance.HIGH, "docs"); http://git-wip-us.apache.org/repos/asf/kafka/blob/d4c4bcf0/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java ---------------------------------------------------------------------- diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index 9e5342e..fb286e2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -298,8 +298,8 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con boolean required = false; String defaultValue; - if (configKey.defaultValue == ConfigDef.NO_DEFAULT_VALUE) { - defaultValue = (String) configKey.defaultValue; + if (ConfigDef.NO_DEFAULT_VALUE.equals(configKey.defaultValue)) { + defaultValue = null; required = true; } else { defaultValue = ConfigDef.convertToString(configKey.defaultValue, type); http://git-wip-us.apache.org/repos/asf/kafka/blob/d4c4bcf0/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java ---------------------------------------------------------------------- diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index 088b520..7ba6fd2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -104,13 +104,13 @@ public class ConnectorPluginsResourceTest { configs.addAll(result.values()); partialConfigs.addAll(partialResult.values()); - ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, "", "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", Collections.<String>emptyList()); + ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, null, "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", Collections.<String>emptyList()); ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.<String>emptyList(), Collections.<String>emptyList(), true); ConfigInfo configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); partialConfigs.add(configInfo); - configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, "", "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", Collections.<String>emptyList()); + configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, null, "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", Collections.<String>emptyList()); configValueInfo = new ConfigValueInfo("test.int.config", "1", Arrays.asList("1", "2", "3"), Collections.<String>emptyList(), true); configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); @@ -122,7 +122,7 @@ public class ConnectorPluginsResourceTest { configs.add(configInfo); partialConfigs.add(configInfo); - configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, "", "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", Collections.<String>emptyList()); + configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, null, "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", Collections.<String>emptyList()); configValueInfo = new ConfigValueInfo("test.list.config", "a,b", Arrays.asList("a", "b", "c"), Collections.<String>emptyList(), true); configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo);
