[
https://issues.apache.org/jira/browse/FLINK-14462?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16955420#comment-16955420
]
Zhu Zhu edited comment on FLINK-14462 at 10/20/19 7:44 AM:
-----------------------------------------------------------
Thanks [~tison] for pointing out this.
Checked and agreed that {{JobGraph#setAllowQueuedScheduling}} has been always
used as true in production(and also in MiniCluster tests) currently.
Also agreed to remove the related logic respecting {{allowQueuedScheduling}}.
Currently a job would fail if queued scheduling is not allowed, since the slot
provider always has no slot initially.
Just a reminder that to drop {{JobGraph#setAllowQueuedScheduling}}, we may need
to take care of 2 things.
The first is to not break any unit tests which relies on
allowQueuedScheduling==false to pass, as it was false by default.
The second is that some docs needs to be updated as well, like in the [mesos
doc|https://ci.apache.org/projects/flink/flink-docs-release-1.9/ops/deployment/mesos.html#flink-job-cluster-on-mesos]
which requires the user to set allowQueuedScheduling to true when generating a
JobGraph.
was (Author: zhuzh):
Checked and agreed that {{JobGraph#setAllowQueuedScheduling}} has been always
used as true in production(and also in MiniCluster tests) currently.
Also agreed to remove the related logic respecting {{allowQueuedScheduling}}.
Currently a job would fail if queued scheduling is not allowed, since the slot
provider always has no slot initially.
Just a reminder that to drop {{JobGraph#setAllowQueuedScheduling}}, we may need
to take care of 2 things.
The first is to not break any unit tests which relies on
allowQueuedScheduling==false to pass, as it was false by default.
The second is that some docs needs to be updated as well, like in the [mesos
doc|https://ci.apache.org/projects/flink/flink-docs-release-1.9/ops/deployment/mesos.html#flink-job-cluster-on-mesos]
which requires the user to set allowQueuedScheduling to true when generating a
JobGraph.
> Remove JobGraph#allowQueuedScheduling flag because it is always true
> --------------------------------------------------------------------
>
> Key: FLINK-14462
> URL: https://issues.apache.org/jira/browse/FLINK-14462
> Project: Flink
> Issue Type: Sub-task
> Components: Runtime / Configuration
> Affects Versions: 1.10.0
> Reporter: Zili Chen
> Assignee: Zili Chen
> Priority: Major
> Fix For: 1.10.0
>
>
> CC [~trohrmann][~zhuzh]
> The only point {{#setAllowQueuedScheduling(false)}} is in
> {{JobGraphGenerator}}. IIRC we always {{#setAllowQueuedScheduling(true)}}
> after the generation and before the submission. For reduce confusion I
> propose to remove {{JobGraph#allowQueuedScheduling}} and refactor the related
> logic to all respect {{true}}.
> This flag is originally used for configuring different resource allocation
> strategy between legacy mode and FLIP-6 arch. And there remains branches in
> {{Scheduler}} which might cause further confusion.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)