kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ewe...@apache.org
Subject kafka git commit: KAFKA-3654: Config validation should validate both common and connector specific configurations
Date Wed, 04 May 2016 01:05:42 GMT
Repository: kafka
Updated Branches:
  refs/heads/0.10.0 138b37d62 -> 2d9369663


KAFKA-3654: Config validation should validate both common and connector specific configurations

Author: Liquan Pei <liquanpei@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1313 from Ishiihara/config-short-circuit

(cherry picked from commit 50aacd660d4c4212ffa4a9dca7d45bcfe50af833)
Signed-off-by: Ewen Cheslack-Postava <me@ewencp.org>


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/2d936966
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/2d936966
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/2d936966

Branch: refs/heads/0.10.0
Commit: 2d93696634bd2a99858b4ba2578b87dae8b86a8a
Parents: 138b37d
Author: Liquan Pei <liquanpei@gmail.com>
Authored: Tue May 3 18:05:04 2016 -0700
Committer: Ewen Cheslack-Postava <me@ewencp.org>
Committed: Tue May 3 18:05:36 2016 -0700

----------------------------------------------------------------------
 .../kafka/connect/runtime/AbstractHerder.java   |  5 ----
 .../resources/ConnectorPluginsResourceTest.java | 25 +++++++++++++++++---
 2 files changed, 22 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/2d936966/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 83f56e2..ee20859 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
@@ -234,11 +234,6 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener,
Con
     public ConfigInfos validateConfigs(String connType, Map<String, String> connectorConfig)
{
         ConfigDef connectorConfigDef = ConnectorConfig.configDef();
         List<ConfigValue> connectorConfigValues = connectorConfigDef.validate(connectorConfig);
-        ConfigInfos result = generateResult(connType, connectorConfigDef.configKeys(), connectorConfigValues,
Collections.<String>emptyList());
-
-        if (result.errorCount() != 0) {
-            return result;
-        }
 
         Connector connector = getConnector(connType);
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/2d936966/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 241d331..ddf30c7 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
@@ -25,9 +25,11 @@ import org.apache.kafka.common.config.ConfigDef.Importance;
 import org.apache.kafka.common.config.ConfigDef.Recommender;
 import org.apache.kafka.common.config.ConfigDef.Type;
 import org.apache.kafka.common.config.ConfigDef.Width;
+import org.apache.kafka.common.config.ConfigValue;
 import org.apache.kafka.connect.connector.Connector;
 import org.apache.kafka.connect.connector.Task;
 import org.apache.kafka.connect.runtime.AbstractHerder;
+import org.apache.kafka.connect.runtime.ConnectorConfig;
 import org.apache.kafka.connect.runtime.Herder;
 import org.apache.kafka.connect.runtime.rest.RestServer;
 import org.apache.kafka.connect.runtime.rest.entities.ConfigInfo;
@@ -70,16 +72,23 @@ public class ConnectorPluginsResourceTest {
 
     private static Map<String, String> props = new HashMap<>();
     static {
+        props.put("name", "test");
         props.put("test.string.config", "testString");
         props.put("test.int.config", "1");
         props.put("test.list.config", "a,b");
     }
 
     private static final ConfigInfos CONFIG_INFOS;
+    private static final int ERROR_COUNT = 1;
 
     static {
         List<ConfigInfo> configs = new LinkedList<>();
 
+        ConfigDef connectorConfigDef = ConnectorConfig.configDef();
+        List<ConfigValue> connectorConfigValues = connectorConfigDef.validate(props);
+        ConfigInfos result = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(),
connectorConfigDef.configKeys(), connectorConfigValues, Collections.<String>emptyList());
+        configs.addAll(result.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());
         ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString",
Collections.<String>emptyList(), Collections.<String>emptyList(), true);
         ConfigInfo configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
@@ -100,7 +109,7 @@ public class ConnectorPluginsResourceTest {
         configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
         configs.add(configInfo);
 
-        CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(),
0, Collections.singletonList("Test"), configs);
+        CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(),
ERROR_COUNT, Collections.singletonList("Test"), configs);
     }
 
     @Mock
@@ -121,10 +130,20 @@ public class ConnectorPluginsResourceTest {
         PowerMock.expectLastCall().andAnswer(new IAnswer<ConfigInfos>() {
             @Override
             public ConfigInfos answer() {
-                Config config = new ConnectorPluginsResourceTestConnector().validate(props);
+                ConfigDef connectorConfigDef = ConnectorConfig.configDef();
+                List<ConfigValue> connectorConfigValues = connectorConfigDef.validate(props);
+
                 Connector connector = new ConnectorPluginsResourceTestConnector();
+                Config config = connector.validate(props);
                 ConfigDef configDef = connector.config();
-                return AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(),
configDef.configKeys(), config.configValues(), configDef.groups());
+                Map<String, ConfigDef.ConfigKey> configKeys = configDef.configKeys();
+                List<ConfigValue> configValues = config.configValues();
+
+                Map<String, ConfigDef.ConfigKey> resultConfigKeys = new HashMap<>(configKeys);
+                resultConfigKeys.putAll(connectorConfigDef.configKeys());
+                configValues.addAll(connectorConfigValues);
+
+                return AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(),
resultConfigKeys, configValues, Collections.singletonList("Test"));
             }
         });
         PowerMock.replayAll();


Mime
View raw message