Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/292#discussion_r23169866 --- Diff: flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java --- @@ -0,0 +1,429 @@ +/* + * 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.client; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.PosixParser; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; +import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus; +import org.apache.flink.util.InstantiationUtil; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileOutputStream; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; + +/** + * Class handling the command line interface to the YARN session. + */ +public class FlinkYarnSessionCli { + private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnSessionCli.class); + + //------------------------------------ Constants ------------------------- + + private static final String CONFIG_FILE_NAME = "flink-conf.yaml"; + public static final String CONFIG_FILE_LOGBACK_NAME = "logback.xml"; + public static final String CONFIG_FILE_LOG4J_NAME = "log4j.properties"; + + + private static final int CLIENT_POLLING_INTERVALL = 3; + + + //------------------------------------ Command Line argument options ------------------------- + // the prefix transformation is used by the CliFrontend static constructor. + private final Option QUERY; + // --- or --- + private final Option QUEUE; + private final Option SHIP_PATH; + private final Option FLINK_JAR; + private final Option JM_MEMORY; + private final Option TM_MEMORY; + private final Option CONTAINER; + private final Option SLOTS; + + /** + * Dynamic properties allow the user to specify additional configuration values with -D, such as + * -Dfs.overwrite-files=true -Dtaskmanager.network.numberOfBuffers=16368 + */ + private final Option DYNAMIC_PROPERTIES; + + private AbstractFlinkYarnCluster yarnCluster = null; + + public FlinkYarnSessionCli(String shortPrefix, String longPrefix) { + QUERY = new Option(shortPrefix + "q", longPrefix + "query", false, "Display available YARN resources (memory, cores)"); + QUEUE = new Option(shortPrefix + "qu", longPrefix + "queue", true, "Specify YARN queue."); + SHIP_PATH = new Option(shortPrefix + "t", longPrefix + "ship", true, "Ship files in the specified directory (t for transfer)"); + FLINK_JAR = new Option(shortPrefix + "j", longPrefix + "jar", true, "Path to Flink jar file"); + JM_MEMORY = new Option(shortPrefix + "jm", longPrefix + "jobManagerMemory", true, "Memory for JobManager Container [in MB]"); + TM_MEMORY = new Option(shortPrefix + "tm", longPrefix + "taskManagerMemory", true, "Memory per TaskManager Container [in MB]"); + CONTAINER = new Option(shortPrefix + "n", longPrefix + "container", true, "Number of YARN container to allocate (=Number of Task Managers)"); + SLOTS = new Option(shortPrefix + "s", longPrefix + "slots", true, "Number of slots per TaskManager"); + DYNAMIC_PROPERTIES = new Option(shortPrefix + "D", true, "Dynamic properties"); + } + + public AbstractFlinkYarnClient createFlinkYarnClient(CommandLine cmd) { + + AbstractFlinkYarnClient flinkYarnClient = getFlinkYarnClient(); + if(flinkYarnClient == null) { + return null; + } + + if(!cmd.hasOption(CONTAINER.getOpt())) { // number of containers is required option! + LOG.error("Missing required argument " + CONTAINER.getOpt()); + printUsage(); + return null; + } + flinkYarnClient.setTaskManagerCount(Integer.valueOf(cmd.getOptionValue(CONTAINER.getOpt()))); + + // Jar Path + Path localJarPath; + if(cmd.hasOption(FLINK_JAR.getOpt())) { + String userPath = cmd.getOptionValue(FLINK_JAR.getOpt()); + if(!userPath.startsWith("file://")) { + userPath = "file://" + userPath; + } + localJarPath = new Path(userPath); + } else { + LOG.info("No path for the flink jar passed. Using the location of "+flinkYarnClient.getClass()+" to locate the jar"); + localJarPath = new Path("file://"+flinkYarnClient.getClass().getProtectionDomain().getCodeSource().getLocation().getPath()); + if(!localJarPath.toString().contains("uberjar")) { + // we need to have a proper uberjar because otherwise we don't have the required classes available on the cluster. + // most likely the user did try to start yarn in a regular hadoop2 flink build (not a yarn package) (using ./bin/flink -m yarn-cluster) + LOG.error("The detected jar file '"+localJarPath+"' is not a uberjar."); + return null; + } + } + + flinkYarnClient.setLocalJarPath(localJarPath); + + // Conf Path + String confDirPath = CliFrontend.getConfigurationDirectoryFromEnv(); + GlobalConfiguration.loadConfiguration(confDirPath); + flinkYarnClient.setConfigurationDirectory(confDirPath); + File confFile = new File(confDirPath + File.separator + CONFIG_FILE_NAME); + if(!confFile.exists()) { + LOG.error("Unable to locate configuration file in "+confFile); + return null; + } + Path confPath = new Path(confFile.getAbsolutePath()); + + flinkYarnClient.setConfigurationFilePath(confPath); + + List<File> shipFiles = new ArrayList<File>(); + // path to directory to ship + if(cmd.hasOption(SHIP_PATH.getOpt())) { + String shipPath = cmd.getOptionValue(SHIP_PATH.getOpt()); + File shipDir = new File(shipPath); + if(shipDir.isDirectory()) { + shipFiles = new ArrayList<File>(Arrays.asList(shipDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return !(name.equals(".") || name.equals("..")); + } + }))); + } else { + LOG.warn("Ship directory is not a directory. Ignoring it."); + } + } + + //check if there is a logback or log4j file + if(confDirPath.length() > 0) { + File logback = new File(confDirPath + File.pathSeparator + CONFIG_FILE_LOGBACK_NAME); + if(logback.exists()) { + shipFiles.add(logback); + flinkYarnClient.setConfigurationFilePath(new Path(logback.toURI())); + } + File log4j = new File(confDirPath + File.pathSeparator + CONFIG_FILE_LOG4J_NAME); + if(log4j.exists()) { + shipFiles.add(log4j); + if(flinkYarnClient.getFlinkLoggingConfigurationPath() != null) { + // this means there is already a logback configuration file --> fail + LOG.error("The configuration directory ('"+confDirPath+"') contains both LOG4J and Logback configuration files." + + "Please delete or rename one of them."); + return null; + } // else + flinkYarnClient.setConfigurationFilePath(new Path(log4j.toURI())); + } + } --- End diff -- Can we check that the corresponding logging backend will actually be used by Flink? Because the system should rather pick the config file whose backend is present.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---