@Test public void testDynamicProperties() throws IOException { Map<String, String> map = new HashMap<String, String>(System.getenv()); File tmpFolder = tmp.newFolder(); File fakeConf = new File(tmpFolder, "flink-conf.yaml"); fakeConf.createNewFile(); map.put(ConfigConstants.ENV_FLINK_CONF_DIR, tmpFolder.getAbsolutePath()); TestBaseUtils.setEnv(map); FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", "", false); Options options = new Options(); cli.addGeneralOptions(options); cli.addRunOptions(options); CommandLineParser parser = new DefaultParser(); CommandLine cmd = null; try { cmd = parser.parse( options, new String[] {"run", "-j", "fake.jar", "-n", "15", "-D", "akka.ask.timeout=5 min"}); } catch (Exception e) { e.printStackTrace(); Assert.fail("Parsing failed with " + e.getMessage()); } AbstractYarnClusterDescriptor flinkYarnDescriptor = cli.createDescriptor(null, cmd); Assert.assertNotNull(flinkYarnDescriptor); Map<String, String> dynProperties = FlinkYarnSessionCli.getDynamicProperties(flinkYarnDescriptor.getDynamicPropertiesEncoded()); Assert.assertEquals(1, dynProperties.size()); Assert.assertEquals("5 min", dynProperties.get("akka.ask.timeout")); }
@Test public void testCorrectSettingOfMaxSlots() throws Exception { File confFile = tmp.newFile("flink-conf.yaml"); File jarFile = tmp.newFile("test.jar"); new CliFrontend(tmp.getRoot().getAbsolutePath()); String[] params = new String[] {"-yn", "2", "-ys", "3", jarFile.getAbsolutePath()}; RunOptions runOptions = CliFrontendParser.parseRunCommand(params); FlinkYarnSessionCli yarnCLI = new TestCLI("y", "yarn"); AbstractYarnClusterDescriptor descriptor = yarnCLI.createDescriptor("", runOptions.getCommandLine()); // each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased. Assert.assertEquals(3, descriptor.getTaskManagerSlots()); Assert.assertEquals(2, descriptor.getTaskManagerCount()); Configuration config = new Configuration(); CliFrontend.setJobManagerAddressInConfig(config, new InetSocketAddress("test", 9000)); ClusterClient client = new TestingYarnClusterClient(descriptor, config); Assert.assertEquals(6, client.getMaxSlots()); }
@Test public void testNotEnoughTaskSlots() throws Exception { File confFile = tmp.newFile("flink-conf.yaml"); File jarFile = tmp.newFile("test.jar"); new CliFrontend(tmp.getRoot().getAbsolutePath()); String[] params = new String[] {"-yn", "2", "-ys", "3", "-p", "7", jarFile.getAbsolutePath()}; RunOptions runOptions = CliFrontendParser.parseRunCommand(params); FlinkYarnSessionCli yarnCLI = new TestCLI("y", "yarn"); AbstractYarnClusterDescriptor descriptor = yarnCLI.createDescriptor("", runOptions.getCommandLine()); // each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased. Assert.assertEquals(4, descriptor.getTaskManagerSlots()); Assert.assertEquals(2, descriptor.getTaskManagerCount()); }