Repository: flink
Updated Branches:
  refs/heads/master ac3997927 -> e4807621b


[FLINK-4998][yarn] fail if too many task slots are configured

This fails the deployment of the Yarn application if the number of task
slots are configured to be larger than the maximum virtual cores of the
Yarn cluster.

This closes #2741.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/1c0c6684
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/1c0c6684
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/1c0c6684

Branch: refs/heads/master
Commit: 1c0c6684b0616e5e35c08870be436d054d106746
Parents: ac39979
Author: Maximilian Michels <[email protected]>
Authored: Thu Oct 27 16:37:56 2016 +0100
Committer: Maximilian Michels <[email protected]>
Committed: Fri Nov 4 20:07:37 2016 +0100

----------------------------------------------------------------------
 .../yarn/AbstractYarnClusterDescriptor.java     |  9 +++
 .../flink/yarn/YarnClusterDescriptorTest.java   | 69 ++++++++++++++++++++
 2 files changed, 78 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1c0c6684/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
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 55bc387..4219a9e 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
@@ -23,6 +23,7 @@ import org.apache.flink.client.deployment.ClusterDescriptor;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.IllegalConfigurationException;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.runtime.security.SecurityContext;
@@ -300,6 +301,14 @@ public abstract class AbstractYarnClusterDescriptor 
implements ClusterDescriptor
                        throw new YarnDeploymentException("Flink configuration 
object has not been set");
                }
 
+               int numYarnVcores = conf.getInt(YarnConfiguration.NM_VCORES, 
YarnConfiguration.DEFAULT_NM_VCORES);
+               // don't configure more than the maximum configured number of 
vcores
+               if (slots > numYarnVcores) {
+                       throw new IllegalConfigurationException(
+                               String.format("The number of task slots per 
node was configured with %d" +
+                                       " but Yarn only has %d virtual cores 
available.", slots, numYarnVcores));
+               }
+
                // 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) {

http://git-wip-us.apache.org/repos/asf/flink/blob/1c0c6684/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..17ce7be
--- /dev/null
+++ 
b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.yarn;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+
+public class YarnClusterDescriptorTest {
+
+       @Rule
+       public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+       private File flinkJar;
+       private File flinkConf;
+
+       @Before
+       public void beforeTest() throws IOException {
+               temporaryFolder.create();
+               flinkJar = temporaryFolder.newFile("flink.jar");
+               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();
+               } catch (Exception e) {
+                       Assert.assertTrue(e.getCause() instanceof 
IllegalConfigurationException);
+               }
+       }
+
+
+}

Reply via email to