[ 
https://issues.apache.org/jira/browse/KAFKA-5807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16484561#comment-16484561
 ] 

ASF GitHub Bot commented on KAFKA-5807:
---------------------------------------

ewencp closed pull request #3762: KAFKA-5807 - Check Connector.config() and 
Transformation.config() returns a valid ConfigDef
URL: https://github.com/apache/kafka/pull/3762
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java 
b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java
index a8a5dabfc83..30dfd3cc9e0 100644
--- 
a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java
+++ 
b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java
@@ -19,6 +19,7 @@
 import org.apache.kafka.common.config.Config;
 import org.apache.kafka.common.config.ConfigDef;
 import org.apache.kafka.common.config.ConfigValue;
+import org.apache.kafka.connect.errors.ConnectException;
 
 import java.util.List;
 import java.util.Map;
@@ -130,13 +131,18 @@ public void reconfigure(Map<String, String> props) {
      */
     public Config validate(Map<String, String> connectorConfigs) {
         ConfigDef configDef = config();
+        if (null == configDef) {
+            throw new ConnectException(
+                String.format("%s.config() must return a ConfigDef that is not 
null.", this.getClass().getName())
+            );
+        }
         List<ConfigValue> configValues = configDef.validate(connectorConfigs);
         return new Config(configValues);
     }
 
     /**
      * Define the configuration for the connector.
-     * @return The ConfigDef for this connector.
+     * @return The ConfigDef for this connector; may not be null.
      */
     public abstract ConfigDef config();
 }
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 424b4749fe2..c31568664fc 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
@@ -271,7 +271,23 @@ public ConfigInfos validateConnectorConfig(Map<String, 
String> connectorProps) {
 
             // do custom connector-specific validation
             Config config = connector.validate(connectorProps);
+            if (null == config) {
+                throw new BadRequestException(
+                    String.format(
+                        "%s.validate() must return a Config that is not null.",
+                        connector.getClass().getName()
+                    )
+                );
+            }
             ConfigDef configDef = connector.config();
+            if (null == configDef) {
+                throw new BadRequestException(
+                    String.format(
+                        "%s.config() must return a ConfigDef that is not 
null.",
+                        connector.getClass().getName()
+                    )
+                );
+            }
             configKeys.putAll(configDef.configKeys());
             allGroups.addAll(configDef.groups());
             configValues.addAll(config.configValues());
diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
index 6a90310df5a..a8dd49a4091 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java
@@ -227,11 +227,22 @@ static ConfigDef getConfigDefFromTransformation(String 
key, Class<?> transformat
         if (transformationCls == null || 
!Transformation.class.isAssignableFrom(transformationCls)) {
             throw new ConfigException(key, String.valueOf(transformationCls), 
"Not a Transformation");
         }
+        Transformation transformation;
         try {
-            return 
(transformationCls.asSubclass(Transformation.class).newInstance()).config();
+            transformation = 
transformationCls.asSubclass(Transformation.class).newInstance();
         } catch (Exception e) {
             throw new ConfigException(key, String.valueOf(transformationCls), 
"Error getting config definition from Transformation: " + e.getMessage());
         }
+        ConfigDef configDef = transformation.config();
+        if (null == configDef) {
+            throw new ConnectException(
+                String.format(
+                    "%s.config() must return a ConfigDef that is not null.",
+                    transformationCls.getName()
+                )
+            );
+        }
+        return configDef;
     }
 
     /**


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Check Connector.config() and Transformation.config() returns a valid ConfigDef
> ------------------------------------------------------------------------------
>
>                 Key: KAFKA-5807
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5807
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Jeremy Custenborder
>            Assignee: Jeremy Custenborder
>            Priority: Minor
>             Fix For: 2.0.0
>
>
> NPE is thrown when a developer returns a null when overloading 
> Connector.validate(). 
> {code}
> [2017-08-23 13:36:30,086] ERROR Stopping after connector error 
> (org.apache.kafka.connect.cli.ConnectStandalone:99)
> java.lang.NullPointerException
>         at 
> org.apache.kafka.connect.connector.Connector.validate(Connector.java:134)
>         at 
> org.apache.kafka.connect.runtime.AbstractHerder.validateConnectorConfig(AbstractHerder.java:254)
>         at 
> org.apache.kafka.connect.runtime.standalone.StandaloneHerder.putConnectorConfig(StandaloneHerder.java:158)
>         at 
> org.apache.kafka.connect.cli.ConnectStandalone.main(ConnectStandalone.java:93)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to