Laszlo Istvan Hunyady created KAFKA-13306:
---------------------------------------------

             Summary: Null config value passes validation, but fails creation
                 Key: KAFKA-13306
                 URL: https://issues.apache.org/jira/browse/KAFKA-13306
             Project: Kafka
          Issue Type: Bug
            Reporter: Laszlo Istvan Hunyady


When validating a connector config containing a property with a null value the 
validation passes, but when creating a connector with the same config the 
worker fails to start the connector because of an invalid config.

Steps to reproduce:

1., Send PUT request to
{\{connectRest}}/connector-plugins/FileStreamSource/config/validate

Request body:
{
 "connector.class": "FileStreamSource",
 "name": "file-source",
 "topic": "target-topic",
 "file":"/source.txt",
 "tasks.max": "1",
 "foo": null
}

Response:
{
 "name": "FileStreamSource",
 "error_count": 0,
 ...
}

2.a., Send PUT request to
{\{connectRest}}/connectors/file-source/config

Request body:
{
 "connector.class": "FileStreamSource",
 "name": "file-source",
 "topic": "target-topic",
 "file":"/source.txt",
 "tasks.max": "1",
 "foo": null
}

2.b., Send Post request to
{\{connectRest}}/connectors/

Request Body:
{
 "name": "file-source",
 "config": {
 "connector.class": "FileStreamSource",
 "name": "file-source",
 "topic": "target-topic",
 "file": "/source.txt",
 "tasks.max": "1",
 "foo": null
 }
}

Result:
Connector is created but connector fails to start, with below exception that 
indicates an invalid config:

ERROR org.apache.kafka.connect.runtime.WorkerConnector: [file-source|worker] 
WorkerConnector\{id=file-source} Error initializing connector
java.lang.ClassCastException: Non-string value found in original settings for 
key foo: null
 at 
org.apache.kafka.common.config.AbstractConfig.originalsStrings(AbstractConfig.java:234)
 at 
org.apache.kafka.connect.runtime.WorkerConnector.initialize(WorkerConnector.java:77)
 at org.apache.kafka.connect.runtime.Worker.startConnector(Worker.java:256)
 at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.startConnector(DistributedHerder.java:1190)
 at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.processConnectorConfigUpdates(DistributedHerder.java:548)
 at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.tick(DistributedHerder.java:395)
 at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.run(DistributedHerder.java:289)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
 at java.util.concurrent.FutureTask.run(FutureTask.java:266)
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 at java.lang.Thread.run(Thread.java:748)

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to