Chris Egerton created KAFKA-17105:
-------------------------------------

             Summary: Unnecessary connector restarts after being newly created
                 Key: KAFKA-17105
                 URL: https://issues.apache.org/jira/browse/KAFKA-17105
             Project: Kafka
          Issue Type: Bug
          Components: connect
    Affects Versions: 3.7.1, 3.6.2, 3.6.1, 3.7.0, 3.5.2, 3.5.1, 3.6.0, 3.4.1, 
3.5.0, 3.3.2, 3.3.1, 3.2.3, 3.2.2, 3.4.0, 3.2.1, 3.1.2, 3.0.2, 3.3.0, 3.1.1, 
3.2.0, 3.0.1, 3.0.0, 3.1.0, 3.8.0, 3.9.0
            Reporter: Chris Egerton


When a connector is created, it may be restarted unnecessarily immediately 
after it is first started by the worker to which it has been assigned:
 # Connector config is written to the config topic
 # A worker reads the new record from the config topic, and adds the connector 
to its connectorConfigUpdates field (see 
[here|https://github.com/apache/kafka/blob/43676f7612b2155ecada54c61b129d996f58bae2/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L2445])
 # The worker participates in the ensuing rebalance (see 
[here|https://github.com/apache/kafka/blob/43676f7612b2155ecada54c61b129d996f58bae2/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L419])
 and is assigned the connector
 # After the rebalance is over (see 
[here|https://github.com/apache/kafka/blob/43676f7612b2155ecada54c61b129d996f58bae2/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L422]),
 the worker starts all of the connectors it has been newly-assigned (see 
[here|https://github.com/apache/kafka/blob/43676f7612b2155ecada54c61b129d996f58bae2/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L1843]
 and 
[here|https://github.com/apache/kafka/blob/43676f7612b2155ecada54c61b129d996f58bae2/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L1963-L2001])
 # Once finished with that, the worker checks for new connector configs (see 
[here|https://github.com/apache/kafka/blob/43676f7612b2155ecada54c61b129d996f58bae2/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L535])
 and restarts all connectors in the connectorConfigUpdates field (see 
[here|https://github.com/apache/kafka/blob/43676f7612b2155ecada54c61b129d996f58bae2/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L717-L726]).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to