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

Lijie Wang edited comment on FLINK-27236 at 4/14/22 7:24 AM:
-------------------------------------------------------------

Hi [~freeke] 
The timeout check will be removed when [the result of 
JobMasterGateway#offerSlots 
returns|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java#:~:text=if%20(!taskSlotTable.-,markSlotActive,-(allocationId))%20%7B]
 (the future is completed). But in the case of large-scale jobs, the main 
thread of the JobMaster will be busy with deploying tasks (In 1.13, deploying 
tasks is very slow, and the main thread will be blocked for a long time), which 
will cause the result of offer slots to come back late, and the slot has been 
released by TM due to timeout.

The slow deployment of task has been solved in Flink 1.14 (FLINK-16069, 
FLINK-23005, [blob: How We Improved Scheduler Performance for Large-scale 
Jobs|https://flink.apache.org/2022/01/04/scheduler-performance-part-one.html] 
), so I think the default value (10s) of "taskmanager.slot.timeout" should work 
well in Flink 1.14, you can try it.


was (Author: wanglijie95):
Hi [~freeke] 
The timeout check will be removed when [the result of 
JobMasterGateway#offerSlots 
returns|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java#:~:text=acceptedSlotsFuture.whenCompleteAsync(-,handleAcceptedSlotOffers]
 (the future is completed). But in the case of large-scale jobs, the main 
thread of the JobMaster will be busy with deploying tasks (In 1.13, deploying 
tasks is very slow, and the main thread will be blocked for a long time), which 
will cause the result of offer slots to come back late, and the slot has been 
released by TM due to timeout.

The slow deployment of task has been solved in Flink 1.14 (FLINK-16069, 
FLINK-23005, [blob: How We Improved Scheduler Performance for Large-scale 
Jobs|https://flink.apache.org/2022/01/04/scheduler-performance-part-one.html] 
), so I think the default value (10s) of "taskmanager.slot.timeout" should work 
well in Flink 1.14, you can try it.

> No task slot allocated for job in larege-scale job
> --------------------------------------------------
>
>                 Key: FLINK-27236
>                 URL: https://issues.apache.org/jira/browse/FLINK-27236
>             Project: Flink
>          Issue Type: Bug
>    Affects Versions: 1.13.3
>            Reporter: yanpengshi
>            Priority: Major
>         Attachments: jobmanager.log.26, taskmanager.log, topology.png
>
>   Original Estimate: 444h
>  Remaining Estimate: 444h
>
> Hey,
>  
> We run a large-scale flink job containing six vertices with 3k parallelism. 
> The Topology is shown below.
> !topology.png!
> We meets the following exception in jobmanager.log:[^jobmanager.log.26]
> {code:java}
> 2022-03-02 08:01:16,601 INFO  [1998] 
> [org.apache.flink.runtime.executiongraph.Execution.transitionState(Execution.java:1446)]
>   - Source: tdbank_exposure_wx -> Flat Map (772/3000) 
> (6cd18d4ead1887a4e19fd3f337a6f4f8) switched from DEPLOYING to FAILED on 
> container_e03_1639558254334_10048_01_004716 @ 11.104.77.40 
> (dataPort=39313).java.util.concurrent.CompletionException: 
> org.apache.flink.runtime.taskexecutor.exceptions.TaskSubmissionException: No 
> task slot allocated for job ID 000000000000ed780000000000000087 and 
> allocation ID beb058d837c09e8d5a4a6aaf2426ca99. {code}
>  
> In the taskmanager.log [^taskmanager.log], the slot is freed due to timeout 
> and the taskmanager receives the new allocated request. By increasing the 
> value of key: taskmanager.slot.timeout, we can avoid this exception 
> temporarily.
> Here are some our guesses:
>  # When the job is scheduled, the slot and execution have been bound, and 
> then the task is deployed to the corresponding taskmanager.
>  # The slot is released after the idle interval times out and notify the 
> ResouceManager the slot free. Thus, the resourceManager will assign other 
> request to the slot.
>  # The task is deployed to taskmanager according the previous correspondence
>  
> The key problems are :
>  # When the slot is free, the execution is not unassigned from the slot;
>  # The slot state is not consistent in JobMaster and ResourceManager
>  
> Has anyone else encountered this problem? When the slot is freed, how can we 
> unassign the previous bounded execution? Or we need to update the resource 
> address of the execution. @[~zhuzh] @[~wanglijie95] 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to