Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-16592

ConfigKey constructor update can break clients using it

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.8.0
    • None

    Description

      In KAFKA-14957, the constructor of ConfigDef.ConfigKey was updated to add a new argument called alternativeString. As part of the PR, new define methods were also added which makes sense. However, since the constructor of ConfigDef.ConfigKey itself can be used directly by other clients which import the dependency, this can break all clients who were using the older constructor w/o the alternativeString argument. 

      I bumped into this when I was testing the[kafka-connect-redis|https://github.com/jcustenborder/kafka-connect-redis/tree/master] connector. It starts up correctly against the official 3.7 release, but fails with the following error when run against a 3.8 snapshot

       

       

      Caused by: java.lang.NoSuchMethodError: org.apache.kafka.common.config.ConfigDef$ConfigKey.<init>(Ljava/lang/String;Lorg/apache/kafka/common/config/ConfigDef$Type;Ljava/lang/Object;Lorg/apache/kafka/common/config/ConfigDef$Validator;Lorg/apache/kafka/common/config/ConfigDef$Importance;Ljava/lang/String;Ljava/lang/String;ILorg/apache/kafka/common/config/ConfigDef$Width;Ljava/lang/String;Ljava/util/List;Lorg/apache/kafka/common/config/ConfigDef$Recommender;Z)V
       at com.github.jcustenborder.kafka.connect.utils.config.ConfigKeyBuilder.build(ConfigKeyBuilder.java:62)
       at com.github.jcustenborder.kafka.connect.redis.RedisConnectorConfig.config(RedisConnectorConfig.java:133)
       at com.github.jcustenborder.kafka.connect.redis.RedisSinkConnectorConfig.config(RedisSinkConnectorConfig.java:46)
       at com.github.jcustenborder.kafka.connect.redis.RedisSinkConnector.config(RedisSinkConnector.java:73)
       at org.apache.kafka.connect.runtime.AbstractHerder.validateConnectorConfig(AbstractHerder.java:538)
       at org.apache.kafka.connect.runtime.AbstractHerder.lambda$validateConnectorConfig$3(AbstractHerder.java:412)
       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)
       ... 1 more
       
      

       

      The reason for that is that the connector uses another library called connect-utils which invokes the old constructor directly

      It is not expected for connector invocations to fail across versions so this would cause confusion.

      We could argue that why is the constructor being invoked directly instead of using the define method, but there might be other clients doing the same. We should add the old constructor back which calls the new one by setting the alternativeString to null.

      Attachments

        Issue Links

          Activity

            People

              sagarrao Sagar Rao
              sagarrao Sagar Rao
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: