Skip to content

Commit

Permalink
[FLINK-4998][yarn] fail if too many task slots are configured
Browse files Browse the repository at this point in the history
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 apache#2741.
  • Loading branch information
mxm committed Nov 4, 2016
1 parent ac39979 commit 1c0c668
Show file tree
Hide file tree
Showing 2 changed files with 78 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
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;
Expand Down Expand Up @@ -300,6 +301,14 @@ private void isReadyForDeployment() throws YarnDeploymentException {
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) {
Expand Down
Original file line number Diff line number Diff line change
@@ -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);
}
}


}

0 comments on commit 1c0c668

Please sign in to comment.