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

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

azagrebin commented on a change in pull request #7255: [FLINK-10945] Use 
InputDependencyConstraint to avoid resource dead…
URL: https://github.com/apache/flink/pull/7255#discussion_r240313527
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
 ##########
 @@ -726,6 +728,56 @@ void sendPartitionInfos() {
                }
        }
 
+       /**
+        * Check whether the InputDependencyConstraint is satisfied for this 
vertex.
+        *
+        * @return whether the input constraint is satisfied
+        */
+       public boolean checkInputDependencyConstraints() {
 
 Review comment:
   This might be a bit more concise:
   ```
      boolean checkInputDependencyConstraints() {
                if (getExecutionGraph().getInputDependencyConstraint() == 
InputDependencyConstraint.ANY) {
                        // InputDependencyConstraint == ANY
                        return 
jobVertex.getInputs().stream().anyMatch(this::isInputConsumable);
                } else {
                        // InputDependencyConstraint == ALL
                        return 
jobVertex.getInputs().stream().allMatch(this::isInputConsumable);
                }
        }
   
        boolean isInputConsumable(IntermediateResult result) {
                if (result.getResultType().isPipelined()) {
                        // For PIPELINED result, the input is consumable if any 
result partition has produced records or is finished
                        return 
Arrays.stream(result.getPartitions()).anyMatch(IntermediateResultPartition::hasDataProduced);
                } else {
                        // For BLOCKING result, the input is consumable if all 
the partitions in the result are finished
                        return result.areAllPartitionsFinished();
                }
        }
   ```
   I am not sure we need to check the `ANY` case at all. Just checking this 
theoretically changes current behaviour. On the other hand, at the moment, I 
think it is always true for `ANY` where we check it if 
`ScheduleMode.LAZY_FROM_SOURCES`. 
   
   I am also not sure that `ALL` config makes sense together with 
`ScheduleMode.EAGER`.

----------------------------------------------------------------
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:
us...@infra.apache.org


> 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