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

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

GJL closed pull request #3614: [FLINK-6189][YARN]Do not use yarn client config 
to do sanity check
URL: https://github.com/apache/flink/pull/3614
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
 
b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index a5a6c364aab..4a12531200c 100644
--- 
a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ 
b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -24,7 +24,6 @@
 import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.configuration.JobManagerOptions;
 import org.apache.flink.configuration.SecurityOptions;
 import org.apache.flink.configuration.TaskManagerOptions;
@@ -318,21 +317,6 @@ private void isReadyForDeployment() throws 
YarnDeploymentException {
                        throw new YarnDeploymentException("Flink configuration 
object has not been set");
                }
 
-               // Check if we don't exceed YARN's maximum virtual cores.
-               // The number of cores can be configured in the config.
-               // If not configured, it is set to the number of task slots
-               int numYarnVcores = conf.getInt(YarnConfiguration.NM_VCORES, 
YarnConfiguration.DEFAULT_NM_VCORES);
-               int configuredVcores = 
flinkConfiguration.getInteger(ConfigConstants.YARN_VCORES, slots);
-               // don't configure more than the maximum configured number of 
vcores
-               if (configuredVcores > numYarnVcores) {
-                       throw new IllegalConfigurationException(
-                               String.format("The number of virtual cores per 
node were configured with %d" +
-                                               " but Yarn only has %d virtual 
cores available. Please note that the number" +
-                                               " of virtual cores is set to 
the number of task slots by default unless configured" +
-                                               " in the Flink config with 
'%s.'",
-                                       configuredVcores, numYarnVcores, 
ConfigConstants.YARN_VCORES));
-               }
-
                // check if required Hadoop environment variables are set. If 
not, warn user
                if(System.getenv("HADOOP_CONF_DIR") == null &&
                        System.getenv("YARN_CONF_DIR") == null) {
diff --git 
a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java 
b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
index a7204da95ee..78e50450237 100644
--- 
a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
+++ 
b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
@@ -50,38 +50,13 @@ public void beforeTest() throws IOException {
                flinkConf = temporaryFolder.newFile("flink-conf.yaml");
        }
 
-       @Test
-       public void testFailIfTaskSlotsHigherThanMaxVcores() {
-
-               YarnClusterDescriptor clusterDescriptor = new 
YarnClusterDescriptor();
-
-               clusterDescriptor.setLocalJarPath(new Path(flinkJar.getPath()));
-               clusterDescriptor.setFlinkConfiguration(new Configuration());
-               
clusterDescriptor.setConfigurationDirectory(temporaryFolder.getRoot().getAbsolutePath());
-               clusterDescriptor.setConfigurationFilePath(new 
Path(flinkConf.getPath()));
-
-               // configure slots too high
-               clusterDescriptor.setTaskManagerSlots(Integer.MAX_VALUE);
-
-               try {
-                       clusterDescriptor.deploy();
-
-                       fail("The deploy call should have failed.");
-               } catch (RuntimeException e) {
-                       // we expect the cause to be an 
IllegalConfigurationException
-                       if (!(e.getCause() instanceof 
IllegalConfigurationException)) {
-                               throw e;
-                       }
-               }
-       }
-
        @Test
        public void testConfigOverwrite() {
 
                YarnClusterDescriptor clusterDescriptor = new 
YarnClusterDescriptor();
 
                Configuration configuration = new Configuration();
-               // overwrite vcores in config
+               // configure slots in config
                configuration.setInteger(ConfigConstants.YARN_VCORES, 
Integer.MAX_VALUE);
 
                clusterDescriptor.setLocalJarPath(new Path(flinkJar.getPath()));
@@ -89,19 +64,10 @@ public void testConfigOverwrite() {
                
clusterDescriptor.setConfigurationDirectory(temporaryFolder.getRoot().getAbsolutePath());
                clusterDescriptor.setConfigurationFilePath(new 
Path(flinkConf.getPath()));
 
-               // configure slots
+               // overwrite vcores
                clusterDescriptor.setTaskManagerSlots(1);
 
-               try {
-                       clusterDescriptor.deploy();
-
-                       fail("The deploy call should have failed.");
-               } catch (RuntimeException e) {
-                       // we expect the cause to be an 
IllegalConfigurationException
-                       if (!(e.getCause() instanceof 
IllegalConfigurationException)) {
-                               throw e;
-                       }
-               }
+               assertEquals(1, clusterDescriptor.getTaskManagerSlots());
        }
 
        @Test


 

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


> Do not use yarn client config to do sanity check
> ------------------------------------------------
>
>                 Key: FLINK-6189
>                 URL: https://issues.apache.org/jira/browse/FLINK-6189
>             Project: Flink
>          Issue Type: Sub-task
>          Components: YARN
>            Reporter: Tao Wang
>            Assignee: Tao Wang
>            Priority: Major
>
> Now in client, if #slots is greater than then number of 
> "yarn.nodemanager.resource.cpu-vcores" in yarn client config, the submission 
> will be rejected.
> It makes no sense as the actual vcores of node manager is decided in cluster 
> side, but not in client side. If we don't set the config or don't set the 
> right value of it(indeed this config is not a mandatory), it should not 
> affect flink submission.



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

Reply via email to