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

Zhanchun Zhang edited comment on FLINK-14511 at 10/25/19 4:09 AM:
------------------------------------------------------------------

 For fair scheduler, we can use `-yqu root.a.product` and `-yqu a.product` to 
submit applications to queue `root.a.product`.

We cannot use `-yqu product` to submit applications to queue `root.a.product`, 
as queue `product` does not exist, and got an exception like this:

{code:java}
2019-10-25 11:36:02,559 WARN org.apache.flink.yarn.YarnClusterDescriptor - The 
specified queue 'product' does not exist. Available queues: root.a, 
root.a.product, root.default, 
2019-10-25 11:36:03,065 WARN org.apache.flink.yarn.YarnClusterDescriptor - The 
configuration directory ('/home/yarn/flink/conf') contains both LOG4J and 
Logback configuration files. Please delete or rename one of them.
2019-10-25 11:36:04,938 INFO org.apache.flink.yarn.YarnClusterDescriptor - 
Submitting application master application_1571906019943_0102
Error while deploying YARN cluster: Couldn't deploy Yarn session cluster
java.lang.RuntimeException: Couldn't deploy Yarn session cluster
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.deploySessionCluster(AbstractYarnClusterDescriptor.java:389)
at 
org.apache.flink.yarn.cli.FlinkYarnSessionCli.run(FlinkYarnSessionCli.java:839)
at 
org.apache.flink.yarn.cli.FlinkYarnSessionCli$1.call(FlinkYarnSessionCli.java:627)
at 
org.apache.flink.yarn.cli.FlinkYarnSessionCli$1.call(FlinkYarnSessionCli.java:624)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1867)
at 
org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
at 
org.apache.flink.yarn.cli.FlinkYarnSessionCli.main(FlinkYarnSessionCli.java:624)
Caused by: org.apache.hadoop.yarn.exceptions.YarnException: Failed to submit 
application_1571906019943_0102 to YARN : Application rejected by queue 
placement policy, queue product does not exist.
at 
org.apache.hadoop.yarn.client.api.impl.YarnClientImpl.submitApplication(YarnClientImpl.java:274)
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.startAppMaster(AbstractYarnClusterDescriptor.java:1013)
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.deployJm(AbstractYarnClusterDescriptor.java:452)
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.deployInternal(AbstractYarnClusterDescriptor.java:418)
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.deploySessionCluster(AbstractYarnClusterDescriptor.java:384)
... 8 more
{code}


If we have a `root.a.product` queue in YARN cluster with FairScheduler,we can 
use `-yqu root.a.product` and `-yqu a.product` to submit applications to the 
cluster successfully, and cannot use `product` to submit applications as it not 
exist.

But if we use `-yqu a.product` to submit applications will get an warning as 
follows, because the queues gettting from `yarnClient.getAllQueues()` method 
are have a `root` prefix for FairScheduer like `root.a, root.a.product`, and 
leaf queue name for CapacityScheduler like `a, product`.

So, I think we should check `root` prefix before checking queues.

{code:java}
2019-10-25 12:04:20,711 WARN org.apache.flink.yarn.YarnClusterDescriptor - The 
specified queue 'a.product' does not exist. Available queues: root.a, 
root.a.product, root.default, ...
{code}



was (Author: dillon.):
 For fair scheduler, we can use `-yqu root.a.product` and `-yqu a.product` to 
submit applications to queue `root.a.product`.

We cannot use `-yqu product` to submit applications to queue `root.a.product`, 
as queue `product` does not exist, and got an exception like this:

{code:java}
2019-10-25 11:36:02,559 WARN org.apache.flink.yarn.YarnClusterDescriptor - The 
specified queue 'product' does not exist. Available queues: root.a, 
root.a.product, root.default, 
2019-10-25 11:36:03,065 WARN org.apache.flink.yarn.YarnClusterDescriptor - The 
configuration directory ('/home/yarn/flink/1200-test/flink-1.4.2-1200/conf') 
contains both LOG4J and Logback configuration files. Please delete or rename 
one of them.
2019-10-25 11:36:04,938 INFO org.apache.flink.yarn.YarnClusterDescriptor - 
Submitting application master application_1571906019943_0102
Error while deploying YARN cluster: Couldn't deploy Yarn session cluster
java.lang.RuntimeException: Couldn't deploy Yarn session cluster
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.deploySessionCluster(AbstractYarnClusterDescriptor.java:389)
at 
org.apache.flink.yarn.cli.FlinkYarnSessionCli.run(FlinkYarnSessionCli.java:839)
at 
org.apache.flink.yarn.cli.FlinkYarnSessionCli$1.call(FlinkYarnSessionCli.java:627)
at 
org.apache.flink.yarn.cli.FlinkYarnSessionCli$1.call(FlinkYarnSessionCli.java:624)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1867)
at 
org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
at 
org.apache.flink.yarn.cli.FlinkYarnSessionCli.main(FlinkYarnSessionCli.java:624)
Caused by: org.apache.hadoop.yarn.exceptions.YarnException: Failed to submit 
application_1571906019943_0102 to YARN : Application rejected by queue 
placement policy, queue product does not exist.
at 
org.apache.hadoop.yarn.client.api.impl.YarnClientImpl.submitApplication(YarnClientImpl.java:274)
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.startAppMaster(AbstractYarnClusterDescriptor.java:1013)
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.deployJm(AbstractYarnClusterDescriptor.java:452)
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.deployInternal(AbstractYarnClusterDescriptor.java:418)
at 
org.apache.flink.yarn.AbstractYarnClusterDescriptor.deploySessionCluster(AbstractYarnClusterDescriptor.java:384)
... 8 more
{code}


If we have a `root.a.product` queue in YARN cluster with FairScheduler,we can 
use `-yqu root.a.product` and `-yqu a.product` to submit applications to the 
cluster successfully, and cannot use `product` to submit applications as it not 
exist.

But if we use `-yqu a.product` to submit applications will get an warning as 
follows, because the queues gettting from `yarnClient.getAllQueues()` method 
are have a `root` prefix for FairScheduer like `root.a, root.a.product`, and 
leaf queue name for CapacityScheduler like `a, product`.

So, I think we should check `root` prefix before checking queues.

{code:java}
2019-10-25 12:04:20,711 WARN org.apache.flink.yarn.YarnClusterDescriptor - The 
specified queue 'a.product' does not exist. Available queues: root.a, 
root.a.product, root.default, ...
{code}


> 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)

Reply via email to