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

ASF GitHub Bot commented on FLINK-10945:
----------------------------------------

zhuzhurk opened a new pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255
 
 
   …locks in LAZY_FROM_SOURCES scheduling when resources are limited
   
   ## What is the purpose of the change
   
   This PR add a job config InputDependencyConstraint, which helps to avoid 
resource deadlocks in LAZY_FROM_SOURCES scheduling when resources are limited, 
as described in 
[FLINK-10945](https://issues.apache.org/jira/browse/FLINK-10945).
   
   ## Brief change log
   
    - *Add InputDependencyConstraint to ExecutionConfig*
    - *Adjust isConsumable interface in IntermediateResultPartition to fit for 
the data actual consumable definition*
    - Change current execution lazy scheduling logic(in 
Execution.scheduleOrUpdateConsumers(edges)) to schedule tasks only if the 
InputDependencyConstraint is satisfied(an interface 
ExecutionVertex.checkInputDependencyConstraints is added to serve this 
purpose). 
   
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - *Added IntermediateResultPartitionTest to validate 
IntermediateResultPartition changes*
     - *Added ExecutionVertexInputConstraintTest to validate the constraint 
check logic in ExecutionVertex*
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes)
     - If yes, how is the feature documented? (JavaDocs)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Avoid resource deadlocks for finite stream jobs when resources are limited
> --------------------------------------------------------------------------
>
>                 Key: FLINK-10945
>                 URL: https://issues.apache.org/jira/browse/FLINK-10945
>             Project: Flink
>          Issue Type: Improvement
>          Components: Distributed Coordination
>    Affects Versions: 1.7.1
>            Reporter: Zhu Zhu
>            Assignee: Zhu Zhu
>            Priority: Major
>              Labels: pull-request-available
>
> Currently *resource deadlocks* can happen to finite stream jobs(or batch 
> jobs) when resources are limited. In 2 cases as below:
>  # Task Y is a pipelined downstream task of task X. When X takes all 
> resources(slots), Y cannot acquire slots to start, thus the back pressure 
> will block X to finish
>  # Task Y is a upstream task of task X. When X takes all resources(slots) and 
> Y cannot start, X cannot finish as some of its inputs are not finished
>  
> We can avoid case 1 by setting all edges to be BLOCKING to avoid pipeline 
> back pressure. However, case 2 cannot be avoided as X(downstream task) will 
> be launched when any of its input result is ready.
> To be detailed, say task X has BLOCKING upstream task Y and Z, X can be 
> launched when Z finishes, though task Y is not launched yet. This pre-launch 
> behaviour can be beneficial when there are plenty of resources, thus X can 
> process data from Z earlier before Y finishes its data processing. However, 
> resource deadlocks may happen when the resources are limited, e.g. in small 
> sessions.
>  
> I’d propose introducing a constraint named as *InputDependencyConstraint* to 
> control the scheduling of vertices. It has 2 values:
>  # *ANY*. The vertex can be scheduled when any of its inputs is consumable.
>  # *ALL*. The vertex can be scheduled when all of its inputs are consumable.
>  
> The design doc is here. 
> [https://docs.google.com/document/d/1jpqC7OW_nLOSVOg06_QCWelicVtV6Au0Krg5m_S4kjY/edit?usp=sharing]



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

Reply via email to