[ https://issues.apache.org/jira/browse/KAFKA-7620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16687257#comment-16687257 ]
ASF GitHub Bot commented on KAFKA-7620: --------------------------------------- rayokota opened a new pull request #5914: KAFKA-7620: Fix restart logic for TTLs URL: https://github.com/apache/kafka/pull/5914 The restart logic for TTLs in `WorkerConfigTransformer` was broken when trying to make it toggle-able. Accessing the toggle through the `Herder` causes the same code to be called recursively. This fix just accesses the toggle by simply looking in the properties map that is passed to `WorkerConfigTransformer`. ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) ---------------------------------------------------------------- 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 > ConfigProvider is broken for KafkaConnect when TTL is not null > -------------------------------------------------------------- > > Key: KAFKA-7620 > URL: https://issues.apache.org/jira/browse/KAFKA-7620 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect > Affects Versions: 2.0.0, 2.0.1 > Reporter: Ye Ji > Assignee: Robert Yokota > Priority: Major > > If the ConfigData returned by ConfigProvider.get implementations has non-null > and non-negative ttl, it will trigger infinite recursion, here is an excerpt > of the stack trace: > {code:java} > at > org.apache.kafka.connect.runtime.WorkerConfigTransformer.scheduleReload(WorkerConfigTransformer.java:62) > at > org.apache.kafka.connect.runtime.WorkerConfigTransformer.scheduleReload(WorkerConfigTransformer.java:56) > at > org.apache.kafka.connect.runtime.WorkerConfigTransformer.transform(WorkerConfigTransformer.java:49) > at > org.apache.kafka.connect.runtime.distributed.ClusterConfigState.connectorConfig(ClusterConfigState.java:121) > at > org.apache.kafka.connect.runtime.distributed.DistributedHerder.connectorConfigReloadAction(DistributedHerder.java:648) > at > org.apache.kafka.connect.runtime.WorkerConfigTransformer.scheduleReload(WorkerConfigTransformer.java:62) > at > org.apache.kafka.connect.runtime.WorkerConfigTransformer.scheduleReload(WorkerConfigTransformer.java:56) > at > org.apache.kafka.connect.runtime.WorkerConfigTransformer.transform(WorkerConfigTransformer.java:49) > {code} > Basically, > 1) if a non-null ttl is returned from the config provider, connect runtime > will try to schedule a reload in the future, > 2) scheduleReload function reads the config again to see if it is a restart > or not, by calling > org.apache.kafka.connect.runtime.WorkerConfigTransformer.transform to > transform the config > 3) the transform function calls config provider, and gets a non-null ttl, > causing scheduleReload being called, we are back to step 1. > To reproduce, simply fork the provided > [FileConfigProvider|https://github.com/apache/kafka/blob/3cdc78e6bb1f83973a14ce1550fe3874f7348b05/clients/src/main/java/org/apache/kafka/common/config/provider/FileConfigProvider.java], > and add a non-negative ttl to the ConfigData returned by the get functions. -- This message was sent by Atlassian JIRA (v7.6.3#76005)