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

Kafka Connect loads old tasks from removed connectors

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.5.1, 3.6.1, 3.8.0
    • 3.8.0, 3.7.1
    • connect
    • None

    Description

      Hello,

      When creating connector we faced an error from one of our ConfigProviders about not existing resource, but we didn't try to set that resource as config value:

      [2024-05-24T12:08:24.362][ERROR][request_id= ][tenant_id= ][thread=DistributedHerder-connect-1-1][class=org.apache.kafka.connect.runtime.distributed.DistributedHerder][method=lambda$reconfigureConnectorTasksWithExponentialBackoffRetries$44] [Worker clientId=connect-1, groupId=streaming-service_streaming_service] Failed to reconfigure connector's tasks (local-file-sink), retrying after backoff.
      org.apache.kafka.common.config.ConfigException: Could not read properties from file /opt/kafka/provider.properties
       at org.apache.kafka.common.config.provider.FileConfigProvider.get(FileConfigProvider.java:98)
       at org.apache.kafka.common.config.ConfigTransformer.transform(ConfigTransformer.java:103)
       at org.apache.kafka.connect.runtime.WorkerConfigTransformer.transform(WorkerConfigTransformer.java:58)
       at org.apache.kafka.connect.storage.ClusterConfigState.taskConfig(ClusterConfigState.java:181)
       at org.apache.kafka.connect.runtime.AbstractHerder.taskConfigsChanged(AbstractHerder.java:804)
       at org.apache.kafka.connect.runtime.distributed.DistributedHerder.publishConnectorTaskConfigs(DistributedHerder.java:2089)
       at org.apache.kafka.connect.runtime.distributed.DistributedHerder.reconfigureConnector(DistributedHerder.java:2082)
       at org.apache.kafka.connect.runtime.distributed.DistributedHerder.reconfigureConnectorTasksWithExponentialBackoffRetries(DistributedHerder.java:2025)
       at org.apache.kafka.connect.runtime.distributed.DistributedHerder.lambda$null$42(DistributedHerder.java:2038)
       at org.apache.kafka.connect.runtime.distributed.DistributedHerder.runRequest(DistributedHerder.java:2232)
       at org.apache.kafka.connect.runtime.distributed.DistributedHerder.tick(DistributedHerder.java:470)
       at org.apache.kafka.connect.runtime.distributed.DistributedHerder.run(DistributedHerder.java:371)
       at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
       at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
       at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
       at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
       at java.base/java.lang.Thread.run(Thread.java:840)
       

      It looked like there already was connector with the same name and same config, but it wasn't.

      After investigation we found out, that few months ago on that cloud there was the connector with the same name and another value for config provider. Then it was removed, but by some reason when we tried to create connector with the same name months ago AbstractHerder tried to update tasks from our previous connector

      As an example I used FileConfigProvider, but actually any ConfigProvider is acceptable which could raise exception if something wrong with config (like result doesn't exist).

      We continued our investigation and found the issue https://issues.apache.org/jira/browse/KAFKA-7745 that says Connect doesn't send tombstone message for commit and task records in the config topic of Kafka Connect. As we remember, the config topic is `compact` that means commit and tasks are are always stored (months, years after connector removing) while tombstones for connector messages are cleaned with delete.retention.ms  property. That impacts further connector creations with the same name.

      We didn't investigate reasons in ConfigClusterStore and how to avoid that issue, because would like to ask, probably it's better to fix KAFKA-7745 and send tombstones for commit and task messages as connect does for connector and target messages?

      In the common way the TC looks like:

      1. Create connector with config provider to resource1
      2. Remove connector
      3. Remove resouce1
      4. Wait 2-4 weeks (until config topic being compacted and tombstone messages about config and target connector are removed)
      5. Try to create connector with the same name and config provider to resource2

      I can provide synthetic TC to reproduce that error if needed.

       

      This is linked with https://issues.apache.org/jira/browse/KAFKA-16837 but it's not the same issue.

      As WA we can remove connector one more time, to get tombstone message for connector in config topic.

      Attachments

        Issue Links

          Activity

            People

              ChrisEgerton Chris Egerton
              mrMigles Sergey Ivanov
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: