You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Stephane Maarek (JIRA)" <ji...@apache.org> on 2017/10/04 00:00:02 UTC
[jira] [Created] (KAFKA-6007) Connect can't validate against
transforms in plugins.path
Stephane Maarek created KAFKA-6007:
--------------------------------------
Summary: Connect can't validate against transforms in plugins.path
Key: KAFKA-6007
URL: https://issues.apache.org/jira/browse/KAFKA-6007
Project: Kafka
Issue Type: Bug
Affects Versions: 0.11.0.1
Reporter: Stephane Maarek
Kafka Connect can't validate a custom transformation if placed in plugins path.
Here's the output I get on the validate call:
{code:java}
Invalid value com.mycorp.kafka.transforms.impl.FlattenSinkRecord for configuration transforms.Flat.type: Class com.mycorp.kafka.transforms.impl.FlattenSinkRecord could not be found.
Invalid value null for configuration transforms.Flat.type: Not a Transformation
"recommended_values": [
"com.mycorp.kafka.transforms.Flatten$Key",
"com.mycorp.kafka.transforms.Flatten$Value",
"com.mycorp.kafka.transforms.impl.FlattenSinkRecord",
"org.apache.kafka.connect.transforms.Cast$Key",
"org.apache.kafka.connect.transforms.Cast$Value",
"org.apache.kafka.connect.transforms.ExtractField$Key",
"org.apache.kafka.connect.transforms.ExtractField$Value",
"org.apache.kafka.connect.transforms.Flatten$Key",
"org.apache.kafka.connect.transforms.Flatten$Value",
"org.apache.kafka.connect.transforms.HoistField$Key",
"org.apache.kafka.connect.transforms.HoistField$Value",
"org.apache.kafka.connect.transforms.InsertField$Key",
"org.apache.kafka.connect.transforms.InsertField$Value",
"org.apache.kafka.connect.transforms.MaskField$Key",
"org.apache.kafka.connect.transforms.MaskField$Value",
"org.apache.kafka.connect.transforms.RegexRouter",
"org.apache.kafka.connect.transforms.ReplaceField$Key",
"org.apache.kafka.connect.transforms.ReplaceField$Value",
"org.apache.kafka.connect.transforms.SetSchemaMetadata$Key",
"org.apache.kafka.connect.transforms.SetSchemaMetadata$Value",
"org.apache.kafka.connect.transforms.TimestampConverter$Key",
"org.apache.kafka.connect.transforms.TimestampConverter$Value",
"org.apache.kafka.connect.transforms.TimestampRouter",
"org.apache.kafka.connect.transforms.ValueToKey"],
{code}
As you can see the class appear in the recommended values (!) but can't be picked up on the validate call.
I believe it's because the recommender implements class discovery using plugins:
https://github.com/apache/kafka/blob/trunk/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java#L194
But the class inference itself doesn't:
https://github.com/apache/kafka/blob/trunk/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java#L199
(I'm not an expert in class loading though, just a guess... Unsure how to fix)
A quick fix is to add the transformations in the ClassPath itself, but that defeats the point a bit.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)