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

Ewen Cheslack-Postava commented on KAFKA-5896:
----------------------------------------------

[~rhauch] I don't have a strong opinion. I don't think it will really solve the 
problem since a task not respecting a request to stop probably indicates a 
buggy implementation and it could very easily be doing some blocking operation 
that won't respond to the interrupt request either (or the implementer might 
catch and ignore InterruptedException in their code because they are forced to 
handle it and don't know what to do with it, or probably other buggy 
implementation issues that would cause this to not work). But it doesn't much 
matter if this is included. Really my complaint is that Java's interrupt 
semantics are terrible and try to give the feeling of preemption when it can't 
and therefore leaves a ton of bugs and overpromised underdelivered guarantees 
in its wake.

> Kafka Connect task threads never interrupted
> --------------------------------------------
>
>                 Key: KAFKA-5896
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5896
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>            Reporter: Nick Pillitteri
>            Assignee: Nick Pillitteri
>            Priority: Minor
>
> h2. Problem
> Kafka Connect tasks associated with connectors are run in their own threads. 
> When tasks are stopped or restarted, a flag is set - {{stopping}} - to 
> indicate the task should stop processing records. However, if the thread the 
> task is running in is blocked (waiting for a lock or performing I/O) it's 
> possible the task will never stop.
> I've created a connector specifically to demonstrate this issue (along with 
> some more detailed instructions for reproducing the issue): 
> https://github.com/smarter-travel-media/hang-connector
> I believe this is an issue because it means that a single badly behaved 
> connector (any connector that does I/O without timeouts) can cause the Kafka 
> Connect worker to get into a state where the only solution is to restart the 
> JVM.
> I think, but couldn't reproduce, that this is the cause of this problem on 
> Stack Overflow: 
> https://stackoverflow.com/questions/43802156/inconsistent-connector-state-connectexception-task-already-exists-in-this-work
> h2. Expected Result
> I would expect the Worker to eventually interrupt the thread that the task is 
> running in. In the past across various other libraries, this is what I've 
> seen done when a thread needs to be forcibly stopped.
> h2. Actual Result
> In actuality, the Worker sets a {{stopping}} flag and lets the thread run 
> indefinitely. It uses a timeout while waiting for the task to stop but after 
> this timeout has expired it simply sets a {{cancelled}} flag. This means that 
> every time a task is restarted, a new thread running the task will be 
> created. Thus a task may end up with multiple instances all running in their 
> own threads when there's only supposed to be a single thread.
> h2. Steps to Reproduce
> The problem can be replicated by using the connector available here: 
> https://github.com/smarter-travel-media/hang-connector
> Apologies for how involved the steps are.
> I've created a patch that forcibly interrupts threads after they fail to 
> gracefully shutdown here: 
> https://github.com/smarter-travel-media/kafka/commit/295c747a9fd82ee8b30556c89c31e0bfcce5a2c5
> I've confirmed that this fixes the issue. I can add some unit tests and 
> submit a PR if people agree that this is a bug and interrupting threads is 
> the right fix.
> Thanks!



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to