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

Till Rohrmann commented on FLINK-17560:
---------------------------------------

Thanks for reporting this issue [~josson]. The root problem in your case is 
that the Task is not properly shutting down. From a resource management 
perspective it makes sense that this slot cannot be reused because it still has 
a resource consumer which might not have properly freed the resources.

The {{TaskCancelerWatchDog}} should kill the task after the configured 
{{task.cancellation.timeout}}. Per default it is 3 minutes. But maybe you are 
right that this task is stuck because of a JVM bug. This could explain why the 
{{TaskManager}} process is not being killed by the {{TaskCancelerWatchDog}}.

The thing which worries me a bit is actually that we are offering the slot to 
the new leader (before the JM fails) even though it still contains a {{Task}} 
which has not properly stopped yet. The problem is that the new leader does not 
know about the not yet stopped {{Task}} and will think that the slot is empty. 
Hence, when the {{JM}} deploys a new {{Task}} into this slot, it might exceed 
the resources of this slot (e.g. network buffers, memory, etc.). Hence, maybe 
we should only offer slots to {{JMs}} if they are empty. Once the {{JM}} is 
able to reconcile the state with a slot which can contain running tasks we 
could change this behavior.

> No Slots available exception in Apache Flink Job Manager while Scheduling
> -------------------------------------------------------------------------
>
>                 Key: FLINK-17560
>                 URL: https://issues.apache.org/jira/browse/FLINK-17560
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.8.3
>         Environment: Flink verson 1.8.3
> Session cluster
>            Reporter: josson paul kalapparambath
>            Priority: Major
>         Attachments: jobmgr.log, threaddump-tm.txt, tm.log
>
>
> Set up
> ------
> Flink verson 1.8.3
> Zookeeper HA cluster
> 1 ResourceManager/Dispatcher (Same Node)
> 1 TaskManager
> 4 pipelines running with various parallelism's
> Issue
> ------
> Occationally when the Job Manager gets restarted we noticed that all the 
> pipelines are not getting scheduled. The error that is reporeted by the Job 
> Manger is 'not enough slots are available'. This should not be the case 
> because task manager was deployed with sufficient slots for the number of 
> pipelines/parallelism we have.
> We further noticed that the slot report sent by the taskmanger contains solts 
> filled with old CANCELLED job Ids. I am not sure why the task manager still 
> holds the details of the old jobs. Thread dump on the task manager confirms 
> that old pipelines are not running.
> I am aware of https://issues.apache.org/jira/browse/FLINK-12865. But this is 
> not the issue happening in this case.



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

Reply via email to