[
https://issues.apache.org/jira/browse/FLINK-14511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16958879#comment-16958879
]
Zhanchun Zhang edited comment on FLINK-14511 at 10/24/19 1:11 PM:
------------------------------------------------------------------
Hi [~fly_in_gis], thanks for your reply.
You are right. For capacity scheduler, we could only use a unique leaf queue to
submit applications, such as: 'product'. For Fair scheduler, we could not only
use a full queue name 'root.product', but also 'product' to submit
applications, as Fair scheduler will add 'root' prefix while scheduling
appliations.
The problem is that, if we submit applications with queue name 'product' to a
YARN cluster with Fair scheduler, YarnClusterDescriptor#checkYarnQueues method
will give a warning information, informs that the 'product' queue does exist.
But we still can submit applications to YARN successfully, which is confusing
for users.
As we don't know what kind of scheduler users use, I think that we should check
queues for both schedulers. YarnClusterDescriptor#checkYarnQueues works well
for capacity scheduler, but not works fine for fair scheduler. If we have a
queue named 'root.product' in YARN with fair scheduler, we can use `-yqu
root.product` or `-yqu product` to submit applications to this queue. So, I
think we should add a ’root‘ prefix before checking queues for Fair scheduler.
was (Author: dillon.):
Hi [~fly_in_gis], thanks for your reply.
You are right. For capacity scheduler, we could only use a unique leaf queue to
submit applications, such as: 'product'. For Fair scheduler, we could not only
use a full queue name 'root.product', but also 'product' to submit
applications, as Fair scheduler will add 'root' prefix while scheduling
appliations.
The problem is that, if we submit applications with queue name 'product' to a
YARN cluster with Fair scheduler, YarnClusterDescriptor#checkYarnQueues method
will give a warning information, informs that the 'product' queue does exist.
But we still can submit applications to YARN successfully, which is confusing
for users.
As we don't know what kind of scheduler user uses, I think that we should check
queues for both scheduler. YarnClusterDescriptor#checkYarnQueues works well for
capacity scheduler, but not works fine for fair scheduler. If we have a queue
named 'root.product' in YARN with fair scheduler, we can use `-yqu
root.product` or `-yqu product` to submit applications to this queue. So, I
think we should add a ’root‘ prefix for Fair scheduler.
> Checking YARN queues should add "root" prefix
> ---------------------------------------------
>
> Key: FLINK-14511
> URL: https://issues.apache.org/jira/browse/FLINK-14511
> Project: Flink
> Issue Type: Bug
> Components: Deployment / YARN
> Reporter: Zhanchun Zhang
> Assignee: Zhanchun Zhang
> Priority: Major
> Labels: pull-request-available
> Time Spent: 20m
> Remaining Estimate: 0h
>
> As we all know, all queues in the YARN cluster are children of the "root"
> queue. While submitting an application to "root.product" queue with -qu
> product parameter, the client logs that "The specified queue 'product' does
> not exist. Available queues....". But this queue is exist and we can still
> submit application to YARN cluster, which is confusing for users. So I think
> that when checking queues should add "root." prefix to the queue name.
> {code:java}
> List<QueueInfo> queues = yarnClient.getAllQueues();
> if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are
> queues configured in yarn and for this session.
> boolean queueFound = false;
> for (QueueInfo queue : queues) {
> if (queue.getQueueName().equals(this.yarnQueue) {
> queueFound = true;
> break;
> }
> }
> if (!queueFound) {
> String queueNames = "";
> for (QueueInfo queue : queues) {
> queueNames += queue.getQueueName() + ", ";
> }
> LOG.warn("The specified queue '" + this.yarnQueue + "' does not
> exist. " +
> "Available queues: " + queueNames);
> }
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)