You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2020/02/08 02:37:00 UTC

[jira] [Commented] (KAFKA-9374) Worker can be disabled by blocked connectors

    [ https://issues.apache.org/jira/browse/KAFKA-9374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17032791#comment-17032791 ] 

ASF GitHub Bot commented on KAFKA-9374:
---------------------------------------

C0urante commented on pull request #8069: KAFKA-9374: Make connector interactions asynchronous
URL: https://github.com/apache/kafka/pull/8069
 
 
   [Jira ticket](https://issues.apache.org/jira/browse/KAFKA-9374)
   
   These changes allow herders to continue to function even when a connector they
   are running hangs in its start, stop, initialize, validate, and/or config
   methods.
   
   The main idea is to make these connector interactions asynchronous and accept
   a callback that can be invoked upon the completion (successful or otherwise) of
   these interactions. The distributed herder handles any follow-up logic by adding
   a new herder request to its queue in that callback, which helps preserve some
   synchronization and ordering guarantees provided by the current tick model.
   
   There are several items that still need to be addressed:
   
   1)  The standalone herder has not been updated to utilize the new asynchronous
       connector wrapper API provided by the Worker and AbstractHerder classes
   
   2)  There is a minor TODO in the DistributedHerderTest class regarding the need
       to migrate some testing logic into the AbstractHerderTest class
   
   3)  More significantly, since connector shutdown is now handled asynchronously,
       there are two problems with the current changes:
   
       a - It is possible (even likely) that a new instance of a connector will be
           created before an older instance has been shut down. This is especially
           problematic if a connector claims a shared resource such as a port
           number and could potentially lead to unnecessary connector failure on
           startup.
   
       b - There is no time allocated during shutdown of the herder for its
           connectors to shutdown, which may lead to improper resource cleanup.
   
   Existing unit tests for the distributed herder and worker have been modified to reflect these changes, and a new integration test named `BlockingConnectorTest` has been added to ensure that they work in practice.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Worker can be disabled by blocked connectors
> --------------------------------------------
>
>                 Key: KAFKA-9374
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9374
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>    Affects Versions: 1.0.0, 1.0.1, 1.0.2, 1.1.0, 1.1.1, 2.0.0, 2.0.1, 2.1.0, 2.2.0, 2.1.1, 2.3.0, 2.2.1, 2.2.2, 2.4.0, 2.3.1
>            Reporter: Chris Egerton
>            Assignee: Chris Egerton
>            Priority: Major
>
> If a connector hangs during any of its {{initialize}}, {{start}}, {{stop}}, \{taskConfigs}}, {{taskClass}}, {{version}}, {{config}}, or {{validate}} methods, the worker will be disabled for some types of requests thereafter, including connector creation, connector reconfiguration, and connector deletion.
>  -This only occurs in distributed mode and is due to the threading model used by the [DistributedHerder|https://github.com/apache/kafka/blob/03f763df8a8d9482d8c099806336f00cf2521465/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java] class.- This affects both distributed and standalone mode. Distributed herders perform some connector work synchronously in their {{tick}} thread, which also handles group membership and some REST requests. The majority of the herder methods for the standalone herder are {{synchronized}}, including those for creating, updating, and deleting connectors; as long as one of those methods blocks, all subsequent calls to any of these methods will also be blocked.
>  
> One potential solution could be to treat connectors that fail to start, stop, etc. in time similarly to tasks that fail to stop within the [task graceful shutdown timeout period|https://github.com/apache/kafka/blob/03f763df8a8d9482d8c099806336f00cf2521465/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java#L121-L126] by handling all connector interactions on a separate thread, waiting for them to complete within a timeout, and abandoning the thread (and transitioning the connector to the {{FAILED}} state, if it has been created at all) if that timeout expires.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)