From f9b52a3114a2114e6846091acf3abb294a49615b Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 22 Apr 2016 19:52:54 +0200 Subject: [PATCH] [FLINK-3667] refactor client communication classes - ClusterDescriptor: base interface for cluster deployment descriptors - ClusterDescriptor: YarnClusterDescriptor - ClusterClient: base class for ClusterClients, handles lifecycle of cluster - ClusterClient: shares configuration with the implementations - ClusterClient: StandaloneClusterClient, YarnClusterClient - ClusterClient: remove run methods and enable detached mode via flag - CliFrontend: remove all Yarn specific logic - CliFrontend: remove all cluster setup logic - CustomCommandLine: interface for other cluster implementations - Customcommandline: enables creation of new cluster or resuming from existing - Yarn: move Yarn classes and functionality to the yarn module (yarn properties, yarn interfaces) - Yarn: improve reliability of cluster startup - Yarn Tests: only disable parallel execution of ITCases This closes #1978 --- .../avro/AvroExternalJarProgramITCase.java | 15 +- .../org/apache/flink/client/CliFrontend.java | 359 ++++------------- .../apache/flink/client/RemoteExecutor.java | 9 +- .../flink/client/cli/CliFrontendParser.java | 114 +++++- .../flink/client/cli/CustomCommandLine.java | 57 +++ .../client/deployment/ClusterDescriptor.java | 41 ++ .../{Client.java => ClusterClient.java} | 361 +++++++++++------- .../client/program/ContextEnvironment.java | 12 +- .../program/ContextEnvironmentFactory.java | 18 +- .../client/program/DetachedEnvironment.java | 6 +- .../program/StandaloneClusterClient.java | 98 +++++ .../CliFrontendAddressConfigurationTest.java | 125 +----- .../client/CliFrontendPackageProgramTest.java | 5 +- .../flink/client/CliFrontendRunTest.java | 26 +- .../flink/client/CliFrontendTestUtils.java | 32 +- ...estingClusterClientWithoutActorSystem.java | 55 +++ .../client/program/ClientConnectionTest.java | 2 +- .../flink/client/program/ClientTest.java | 33 +- .../program/ExecutionPlanCreationTest.java | 2 +- .../apache/flink/storm/api/FlinkClient.java | 11 +- .../flink/api/common/JobExecutionResult.java | 3 + .../flink/api/common/JobSubmissionResult.java | 24 +- .../main/flink-bin/conf/log4j-cli.properties | 2 +- .../main/flink-bin/yarn-bin/yarn-session.sh | 2 +- .../DegreesWithExceptionITCase.java | 2 +- .../ReduceOnEdgesWithExceptionITCase.java | 2 +- .../ReduceOnNeighborsWithExceptionITCase.java | 2 +- .../webmonitor/handlers/JarActionHandler.java | 4 +- .../flink/runtime/client/JobClient.java | 17 +- .../clusterframework/ApplicationStatus.java | 1 + .../FlinkResourceManager.java | 2 +- .../messages/GetClusterStatusResponse.java | 2 +- .../runtime/yarn/AbstractFlinkYarnClient.java | 143 ------- .../yarn/AbstractFlinkYarnCluster.java | 123 ------ .../apache/flink/api/scala/FlinkShell.scala | 82 ++-- .../api/scala/ExecutionEnvironment.scala | 2 +- .../ElasticsearchSinkITCase.java | 2 +- .../environment/RemoteStreamEnvironment.java | 9 +- .../environment/StreamContextEnvironment.java | 5 +- .../RemoteEnvironmentITCase.java | 2 +- .../test/misc/AutoParallelismITCase.java | 2 +- .../test/recovery/SimpleRecoveryITCase.java | 2 +- flink-yarn-tests/pom.xml | 15 +- ...iFrontendYarnAddressConfigurationTest.java | 220 +++++++++++ .../flink/yarn/FlinkYarnSessionCliTest.java | 14 +- ...java => TestingYarnClusterDescriptor.java} | 4 +- .../yarn/YARNHighAvailabilityITCase.java | 9 +- .../YARNSessionCapacitySchedulerITCase.java | 6 +- .../flink/yarn/YARNSessionFIFOITCase.java | 20 +- .../org/apache/flink/yarn/YarnTestBase.java | 4 +- ...ava => AbstractYarnClusterDescriptor.java} | 166 ++++---- .../yarn/YarnApplicationMasterRunner.java | 7 +- ...arnCluster.java => YarnClusterClient.java} | 272 +++++++------ ...Client.java => YarnClusterDescriptor.java} | 4 +- .../flink/yarn/cli}/FlinkYarnSessionCli.java | 343 +++++++++++------ .../apache/flink/yarn/ApplicationClient.scala | 8 +- .../org/apache/flink/yarn/YarnMessages.scala | 7 +- 57 files changed, 1567 insertions(+), 1348 deletions(-) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java rename flink-clients/src/main/java/org/apache/flink/client/program/{Client.java => ClusterClient.java} (68%) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java create mode 100644 flink-clients/src/test/java/org/apache/flink/client/TestingClusterClientWithoutActorSystem.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java create mode 100644 flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java rename flink-yarn-tests/src/test/java/org/apache/flink/yarn/{TestingFlinkYarnClient.java => TestingYarnClusterDescriptor.java} (95%) rename flink-yarn/src/main/java/org/apache/flink/yarn/{FlinkYarnClientBase.java => AbstractYarnClusterDescriptor.java} (87%) rename flink-yarn/src/main/java/org/apache/flink/yarn/{FlinkYarnCluster.java => YarnClusterClient.java} (68%) rename flink-yarn/src/main/java/org/apache/flink/yarn/{FlinkYarnClient.java => YarnClusterDescriptor.java} (83%) rename {flink-clients/src/main/java/org/apache/flink/client => flink-yarn/src/main/java/org/apache/flink/yarn/cli}/FlinkYarnSessionCli.java (58%) diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java index ac1007457fa3b..29a7e586d5d22 100644 --- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java +++ b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java @@ -19,19 +19,12 @@ package org.apache.flink.api.avro; import java.io.File; -import java.net.InetAddress; -import org.apache.flink.api.common.Plan; -import org.apache.flink.client.CliFrontend; -import org.apache.flink.client.RemoteExecutor; -import org.apache.flink.client.program.Client; -import org.apache.flink.client.program.JobWithJars; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.FlinkPlan; -import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.junit.Assert; @@ -64,10 +57,10 @@ public void testExternalProgram() { config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost"); config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, testMiniCluster.getLeaderRPCPort()); - Client client = new Client(config); + ClusterClient client = new StandaloneClusterClient(config); client.setPrintStatusDuringExecution(false); - client.runBlocking(program, 4); + client.run(program, 4); } catch (Throwable t) { diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index 6d972bc5ab426..cf7a8c25f6baa 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -20,8 +20,6 @@ import akka.actor.ActorSystem; -import org.apache.commons.cli.CommandLine; - import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; @@ -31,18 +29,21 @@ import org.apache.flink.client.cli.CliArgsException; import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.CommandLineOptions; +import org.apache.flink.client.cli.CustomCommandLine; import org.apache.flink.client.cli.InfoOptions; import org.apache.flink.client.cli.ListOptions; import org.apache.flink.client.cli.ProgramOptions; import org.apache.flink.client.cli.RunOptions; import org.apache.flink.client.cli.SavepointOptions; import org.apache.flink.client.cli.StopOptions; -import org.apache.flink.client.program.Client; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.optimizer.DataStatistics; import org.apache.flink.optimizer.Optimizer; @@ -53,7 +54,6 @@ import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -68,8 +68,6 @@ import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; import org.apache.flink.util.StringUtils; import org.slf4j.Logger; @@ -81,10 +79,8 @@ import scala.concurrent.duration.FiniteDuration; import java.io.File; -import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InputStream; import java.net.InetSocketAddress; import java.net.URL; import java.text.SimpleDateFormat; @@ -93,10 +89,8 @@ import java.util.Collections; import java.util.Comparator; import java.util.Date; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint; @@ -121,20 +115,6 @@ public class CliFrontend { private static final String CONFIG_DIRECTORY_FALLBACK_1 = "../conf"; private static final String CONFIG_DIRECTORY_FALLBACK_2 = "conf"; - // YARN-session related constants - public static final String YARN_PROPERTIES_FILE = ".yarn-properties-"; - public static final String YARN_PROPERTIES_JOBMANAGER_KEY = "jobManager"; - public static final String YARN_PROPERTIES_PARALLELISM = "parallelism"; - public static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString"; - - public static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; // this has to be a regex for String.split() - - /** - * A special host name used to run a job by deploying Flink into a YARN cluster, - * if this string is specified as the JobManager address - */ - public static final String YARN_DEPLOY_JOBMANAGER = "yarn-cluster"; - // -------------------------------------------------------------------------------------------- // -------------------------------------------------------------------------------------------- @@ -149,12 +129,9 @@ public class CliFrontend { private ActorSystem actorSystem; - private AbstractFlinkYarnCluster yarnCluster; - /** * - * @throws Exception Thrown if the configuration directory was not found, the configuration could not - * be loaded, or the YARN properties could not be parsed. + * @throws Exception Thrown if the configuration directory was not found, the configuration could not be loaded */ public CliFrontend() throws Exception { this(getConfigurationDirectoryFromEnv()); @@ -171,61 +148,6 @@ public CliFrontend(String configDir) throws Exception { GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath()); this.config = GlobalConfiguration.getConfiguration(); - // load the YARN properties - File propertiesFile = new File(getYarnPropertiesLocation(config)); - if (propertiesFile.exists()) { - - logAndSysout("Found YARN properties file " + propertiesFile.getAbsolutePath()); - - Properties yarnProperties = new Properties(); - try { - try (InputStream is = new FileInputStream(propertiesFile)) { - yarnProperties.load(is); - } - } - catch (IOException e) { - throw new Exception("Cannot read the YARN properties file", e); - } - - // configure the default parallelism from YARN - String propParallelism = yarnProperties.getProperty(YARN_PROPERTIES_PARALLELISM); - if (propParallelism != null) { // maybe the property is not set - try { - int parallelism = Integer.parseInt(propParallelism); - this.config.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism); - - logAndSysout("YARN properties set default parallelism to " + parallelism); - } - catch (NumberFormatException e) { - throw new Exception("Error while parsing the YARN properties: " + - "Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer."); - } - } - - // get the JobManager address from the YARN properties - String address = yarnProperties.getProperty(YARN_PROPERTIES_JOBMANAGER_KEY); - InetSocketAddress jobManagerAddress; - if (address != null) { - try { - jobManagerAddress = ClientUtils.parseHostPortAddress(address); - // store address in config from where it is retrieved by the retrieval service - writeJobManagerAddressToConfig(jobManagerAddress); - } - catch (Exception e) { - throw new Exception("YARN properties contain an invalid entry for JobManager address.", e); - } - - logAndSysout("Using JobManager address from YARN properties " + jobManagerAddress); - } - - // handle the YARN client's dynamic properties - String dynamicPropertiesEncoded = yarnProperties.getProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING); - Map dynamicProperties = getDynamicProperties(dynamicPropertiesEncoded); - for (Map.Entry dynamicProperty : dynamicProperties.entrySet()) { - this.config.setString(dynamicProperty.getKey(), dynamicProperty.getValue()); - } - } - try { FileSystem.setDefaultScheme(config); } catch (IOException e) { @@ -301,61 +223,33 @@ protected int run(String[] args) { return handleError(t); } - int exitCode = 1; + ClusterClient client = null; try { - int userParallelism = options.getParallelism(); - LOG.debug("User parallelism is set to {}", userParallelism); - Client client = getClient(options, program.getMainClassName(), userParallelism, options.getDetachedMode()); + client = getClient(options, program.getMainClassName()); client.setPrintStatusDuringExecution(options.getStdoutLogging()); + client.setDetached(options.getDetachedMode()); LOG.debug("Client slots is set to {}", client.getMaxSlots()); LOG.debug("Savepoint path is set to {}", options.getSavepointPath()); - try { - if (client.getMaxSlots() != -1 && userParallelism == -1) { - logAndSysout("Using the parallelism provided by the remote cluster ("+client.getMaxSlots()+"). " + - "To use another parallelism, set it at the ./bin/flink client."); - userParallelism = client.getMaxSlots(); - } - - // detached mode - if (options.getDetachedMode() || (yarnCluster != null && yarnCluster.isDetached())) { - exitCode = executeProgramDetached(program, client, userParallelism); - } - else { - exitCode = executeProgramBlocking(program, client, userParallelism); - } - - // show YARN cluster status if its not a detached YARN cluster. - if (yarnCluster != null && !yarnCluster.isDetached()) { - List msgs = yarnCluster.getNewMessages(); - if (msgs != null && msgs.size() > 1) { - - logAndSysout("The following messages were created by the YARN cluster while running the Job:"); - for (String msg : msgs) { - logAndSysout(msg); - } - } - if (yarnCluster.hasFailed()) { - logAndSysout("YARN cluster is in failed state!"); - logAndSysout("YARN Diagnostics: " + yarnCluster.getDiagnostics()); - } - } - - return exitCode; - } - finally { - client.shutdown(); + int userParallelism = options.getParallelism(); + LOG.debug("User parallelism is set to {}", userParallelism); + if (client.getMaxSlots() != -1 && userParallelism == -1) { + logAndSysout("Using the parallelism provided by the remote cluster (" + + client.getMaxSlots()+"). " + + "To use another parallelism, set it at the ./bin/flink client."); + userParallelism = client.getMaxSlots(); } + + return executeProgram(program, client, userParallelism); } catch (Throwable t) { return handleError(t); } finally { - if (yarnCluster != null && !yarnCluster.isDetached()) { - logAndSysout("Shutting down YARN cluster"); - yarnCluster.shutdown(exitCode != 0); + if (client != null) { + client.shutdown(); } if (program != null) { program.deleteExtractedLibraries(); @@ -410,7 +304,7 @@ protected int info(String[] args) { LOG.info("Creating program plan dump"); Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); - FlinkPlan flinkPlan = Client.getOptimizedPlan(compiler, program, parallelism); + FlinkPlan flinkPlan = ClusterClient.getOptimizedPlan(compiler, program, parallelism); String jsonPlan = null; if (flinkPlan instanceof OptimizedPlan) { @@ -830,53 +724,30 @@ else if (result instanceof DisposeSavepointFailure) { // Interaction with programs and JobManager // -------------------------------------------------------------------------------------------- - protected int executeProgramDetached(PackagedProgram program, Client client, int parallelism) { - LOG.info("Starting execution of program"); + protected int executeProgram(PackagedProgram program, ClusterClient client, int parallelism) { + logAndSysout("Starting execution of program"); JobSubmissionResult result; try { - result = client.runDetached(program, parallelism); + result = client.run(program, parallelism); } catch (ProgramInvocationException e) { return handleError(e); } finally { program.deleteExtractedLibraries(); } - if (yarnCluster != null) { - yarnCluster.stopAfterJob(result.getJobID()); - yarnCluster.disconnect(); - } - - System.out.println("Job has been submitted with JobID " + result.getJobID()); - - return 0; - } - - protected int executeProgramBlocking(PackagedProgram program, Client client, int parallelism) { - LOG.info("Starting execution of program"); - - JobSubmissionResult result; - try { - result = client.runBlocking(program, parallelism); - } - catch (ProgramInvocationException e) { - return handleError(e); - } - finally { - program.deleteExtractedLibraries(); - } - - LOG.info("Program execution finished"); - - if (result instanceof JobExecutionResult) { - JobExecutionResult execResult = (JobExecutionResult) result; + if(result.isJobExecutionResults()) { + logAndSysout("Program execution finished"); + JobExecutionResult execResult = result.getJobExecutionResult(); System.out.println("Job with JobID " + execResult.getJobID() + " has finished."); System.out.println("Job Runtime: " + execResult.getNetRuntime() + " ms"); Map accumulatorsResult = execResult.getAllAccumulatorResults(); if (accumulatorsResult.size() > 0) { - System.out.println("Accumulator Results: "); - System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult)); + System.out.println("Accumulator Results: "); + System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult)); } + } else { + logAndSysout("Job has been submitted with JobID " + result.getJobID()); } return 0; @@ -922,16 +793,6 @@ else if (!jarFile.isFile()) { return program; } - /** - * Writes the given job manager address to the associated configuration object - * - * @param address Address to write to the configuration - */ - protected void writeJobManagerAddressToConfig(InetSocketAddress address) { - config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.getHostName()); - config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.getPort()); - } - /** * Updates the associated configuration with the given command line options * @@ -940,7 +801,7 @@ protected void writeJobManagerAddressToConfig(InetSocketAddress address) { protected void updateConfig(CommandLineOptions options) { if(options.getJobManagerAddress() != null){ InetSocketAddress jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress()); - writeJobManagerAddressToConfig(jobManagerAddress); + writeJobManagerAddressToConfig(config, jobManagerAddress); } } @@ -980,110 +841,65 @@ protected ActorGateway getJobManagerGateway(CommandLineOptions options) throws E } /** - * Retrieves a {@link Client} object from the given command line options and other parameters. + * Retrieves a {@link ClusterClient} object from the given command line options and other parameters. * * @param options Command line options which contain JobManager address * @param programName Program name - * @param userParallelism Given user parallelism * @throws Exception */ - protected Client getClient( + protected ClusterClient getClient( CommandLineOptions options, - String programName, - int userParallelism, - boolean detachedMode) + String programName) throws Exception { InetSocketAddress jobManagerAddress; - int maxSlots = -1; - if (YARN_DEPLOY_JOBMANAGER.equals(options.getJobManagerAddress())) { - logAndSysout("YARN cluster mode detected. Switching Log4j output to console"); + // try to get the JobManager address via command-line args + if (options.getJobManagerAddress() != null) { - // Default yarn application name to use, if nothing is specified on the command line - String applicationName = "Flink Application: " + programName; + // Get the custom command-lines (e.g. Yarn/Mesos) + CustomCommandLine activeCommandLine = + CliFrontendParser.getActiveCustomCommandLine(options.getJobManagerAddress()); - // user wants to run Flink in YARN cluster. - CommandLine commandLine = options.getCommandLine(); - AbstractFlinkYarnClient flinkYarnClient = CliFrontendParser - .getFlinkYarnSessionCli() - .withDefaultApplicationName(applicationName) - .createFlinkYarnClient(commandLine); + if (activeCommandLine != null) { + logAndSysout(activeCommandLine.getIdentifier() + " mode detected. Switching Log4j output to console"); - if (flinkYarnClient == null) { - throw new RuntimeException("Unable to create Flink YARN Client. Check previous log messages"); - } + // Default yarn application name to use, if nothing is specified on the command line + String applicationName = "Flink Application: " + programName; - // in case the main detached mode wasn't set, we don't wanna overwrite the one loaded - // from yarn options. - if (detachedMode) { - flinkYarnClient.setDetachedMode(true); - } + ClusterClient client = activeCommandLine.createClient(applicationName, options.getCommandLine()); - // the number of slots available from YARN: - int yarnTmSlots = flinkYarnClient.getTaskManagerSlots(); - if (yarnTmSlots == -1) { - yarnTmSlots = 1; - } - maxSlots = yarnTmSlots * flinkYarnClient.getTaskManagerCount(); - if (userParallelism != -1) { - int slotsPerTM = userParallelism / flinkYarnClient.getTaskManagerCount(); - logAndSysout("The YARN cluster has " + maxSlots + " slots available, " + - "but the user requested a parallelism of " + userParallelism + " on YARN. " + - "Each of the " + flinkYarnClient.getTaskManagerCount() + " TaskManagers " + - "will get "+slotsPerTM+" slots."); - flinkYarnClient.setTaskManagerSlots(slotsPerTM); - } + logAndSysout("Cluster started"); + logAndSysout("JobManager web interface address " + client.getWebInterfaceURL()); - try { - yarnCluster = flinkYarnClient.deploy(); - yarnCluster.connectToCluster(); - } - catch (Exception e) { - throw new RuntimeException("Error deploying the YARN cluster", e); + return client; + } else { + // job manager address supplied on the command-line + LOG.info("Using address {} to connect to JobManager.", options.getJobManagerAddress()); + jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress()); + writeJobManagerAddressToConfig(config, jobManagerAddress); + return new StandaloneClusterClient(config); } - jobManagerAddress = yarnCluster.getJobManagerAddress(); - writeJobManagerAddressToConfig(jobManagerAddress); - - // overwrite the yarn client config (because the client parses the dynamic properties) - this.config.addAll(flinkYarnClient.getFlinkConfiguration()); - - logAndSysout("YARN cluster started"); - logAndSysout("JobManager web interface address " + yarnCluster.getWebInterfaceURL()); - logAndSysout("Waiting until all TaskManagers have connected"); - - while(true) { - GetClusterStatusResponse status = yarnCluster.getClusterStatus(); - if (status != null) { - if (status.numRegisteredTaskManagers() < flinkYarnClient.getTaskManagerCount()) { - logAndSysout("TaskManager status (" + status.numRegisteredTaskManagers() + "/" - + flinkYarnClient.getTaskManagerCount() + ")"); - } else { - logAndSysout("All TaskManagers are connected"); - break; - } - } else { - logAndSysout("No status updates from the YARN cluster received so far. Waiting ..."); - } - - try { - Thread.sleep(500); - } - catch (InterruptedException e) { - LOG.error("Interrupted while waiting for TaskManagers"); - System.err.println("Thread is interrupted"); - Thread.currentThread().interrupt(); + // try to get the JobManager address via resuming of a cluster + } else { + for (CustomCommandLine cli : CliFrontendParser.getAllCustomCommandLine().values()) { + ClusterClient client = cli.retrieveCluster(config); + if (client != null) { + LOG.info("Using address {} to connect to JobManager.", client.getJobManagerAddressFromConfig()); + return client; } } } - else { - if(options.getJobManagerAddress() != null) { - jobManagerAddress = ClientUtils.parseHostPortAddress(options.getJobManagerAddress()); - writeJobManagerAddressToConfig(jobManagerAddress); - } - } - return new Client(config, maxSlots); + // read JobManager address from the config + if (config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null) != null) { + return new StandaloneClusterClient(config); + // We tried hard but couldn't find a JobManager address + } else { + throw new IllegalConfigurationException( + "The JobManager address is neither provided at the command-line, " + + "nor configured in flink-conf.yaml."); + } } // -------------------------------------------------------------------------------------------- @@ -1275,33 +1091,16 @@ else if (new File(CONFIG_DIRECTORY_FALLBACK_2).exists()) { return location; } - public static Map getDynamicProperties(String dynamicPropertiesEncoded) { - if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) { - Map properties = new HashMap<>(); - - String[] propertyLines = dynamicPropertiesEncoded.split(CliFrontend.YARN_DYNAMIC_PROPERTIES_SEPARATOR); - for (String propLine : propertyLines) { - if (propLine == null) { - continue; - } - - String[] kv = propLine.split("="); - if (kv.length >= 2 && kv[0] != null && kv[1] != null && kv[0].length() > 0) { - properties.put(kv[0], kv[1]); - } - } - return properties; - } - else { - return Collections.emptyMap(); - } - } - - public static String getYarnPropertiesLocation(Configuration conf) { - String defaultPropertiesFileLocation = System.getProperty("java.io.tmpdir"); - String currentUser = System.getProperty("user.name"); - String propertiesFileLocation = conf.getString(ConfigConstants.YARN_PROPERTIES_FILE_LOCATION, defaultPropertiesFileLocation); - return propertiesFileLocation + File.separator + CliFrontend.YARN_PROPERTIES_FILE + currentUser; + /** + * Writes the given job manager address to the associated configuration object + * + * @param address Address to write to the configuration + * @param config The config to write to + */ + public static void writeJobManagerAddressToConfig(Configuration config, InetSocketAddress address) { + config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, address.getHostName()); + config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, address.getPort()); } + } diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java index ab70453b33779..86b36b360e98e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java @@ -27,8 +27,9 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.PlanExecutor; -import org.apache.flink.client.program.Client; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.JobWithJars; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.optimizer.DataStatistics; import org.apache.flink.optimizer.Optimizer; import org.apache.flink.configuration.ConfigConstants; @@ -57,7 +58,7 @@ public class RemoteExecutor extends PlanExecutor { private final Configuration clientConfiguration; - private Client client; + private ClusterClient client; private int defaultParallelism = 1; @@ -149,7 +150,7 @@ public int getDefaultParallelism() { public void start() throws Exception { synchronized (lock) { if (client == null) { - client = new Client(clientConfiguration); + client = new StandaloneClusterClient(clientConfiguration); client.setPrintStatusDuringExecution(isPrintingStatusDuringExecution()); } else { @@ -207,7 +208,7 @@ public JobExecutionResult executePlanWithJars(JobWithJars program) throws Except } try { - return client.runBlocking(program, defaultParallelism); + return client.run(program, defaultParallelism).getJobExecutionResult(); } finally { if (shutDownAtEnd) { diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index b75952e407539..f28d1b67998ef 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -24,8 +24,16 @@ import org.apache.commons.cli.ParseException; import org.apache.commons.cli.PosixParser; -import org.apache.flink.client.CliFrontend; -import org.apache.flink.client.FlinkYarnSessionCli; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + /** * A simple command line parser (based on Apache Commons CLI) that extracts command @@ -33,9 +41,17 @@ */ public class CliFrontendParser { + private static final Logger LOG = LoggerFactory.getLogger(CliFrontendParser.class); + + /** command line interface of the YARN session, with a special initialization here * to prefix all options with y/yarn. */ - private static final FlinkYarnSessionCli yarnSessionCLi = new FlinkYarnSessionCli("y", "yarn", true); + private static final Map customCommandLine = new HashMap<>(1); + + static { + // we could easily add more here in the future + loadCustomCommandLine("org.apache.flink.yarn.cli.FlinkYarnSessionCli", "y", "yarn"); + } static final Option HELP_OPTION = new Option("h", "help", false, @@ -43,7 +59,7 @@ public class CliFrontendParser { static final Option JAR_OPTION = new Option("j", "jarfile", true, "Flink program JAR file."); - public static final Option CLASS_OPTION = new Option("c", "class", true, + static final Option CLASS_OPTION = new Option("c", "class", true, "Class with the program entry point (\"main\" method or \"getPlan()\" method. Only needed if the " + "JAR file does not specify the class in its manifest."); @@ -53,23 +69,23 @@ public class CliFrontendParser { "times for specifying more than one URL. The protocol must be supported by the " + "{@link java.net.URLClassLoader}."); - static final Option PARALLELISM_OPTION = new Option("p", "parallelism", true, + public static final Option PARALLELISM_OPTION = new Option("p", "parallelism", true, "The parallelism with which to run the program. Optional flag to override the default value " + "specified in the configuration."); static final Option LOGGING_OPTION = new Option("q", "sysoutLogging", false, "If present, " + "supress logging output to standard out."); - static final Option DETACHED_OPTION = new Option("d", "detached", false, "If present, runs " + + public static final Option DETACHED_OPTION = new Option("d", "detached", false, "If present, runs " + "the job in detached mode"); static final Option ARGS_OPTION = new Option("a", "arguments", true, "Program arguments. Arguments can also be added without -a, simply as trailing parameters."); static final Option ADDRESS_OPTION = new Option("m", "jobmanager", true, - "Address of the JobManager (master) to which to connect. Specify '" + CliFrontend.YARN_DEPLOY_JOBMANAGER + - "' as the JobManager to deploy a YARN cluster for the job. Use this flag to connect to a " + - "different JobManager than the one specified in the configuration."); + "Address of the JobManager (master) to which to connect. " + + "Specify " + getCliIdentifierString() +" as the JobManager to deploy a cluster for the job. " + + "Use this flag to connect to a different JobManager than the one specified in the configuration."); static final Option SAVEPOINT_PATH_OPTION = new Option("s", "fromSavepoint", true, "Path to a savepoint to reset the job back to (for example file:///flink/savepoint-1537)."); @@ -143,8 +159,10 @@ public static Options getProgramSpecificOptions(Options options) { options.addOption(DETACHED_OPTION); options.addOption(SAVEPOINT_PATH_OPTION); - // also add the YARN options so that the parser can parse them - yarnSessionCLi.getYARNSessionCLIOptions(options); + for (CustomCommandLine customCLI : customCommandLine.values()) { + customCLI.addOptions(options); + } + return options; } @@ -240,10 +258,16 @@ public static void printHelpForRun() { System.out.println("\n Syntax: run [OPTIONS] "); formatter.setSyntaxPrefix(" \"run\" action options:"); formatter.printHelp(" ", getRunOptionsWithoutDeprecatedOptions(new Options())); - formatter.setSyntaxPrefix(" Additional arguments if -m " + CliFrontend.YARN_DEPLOY_JOBMANAGER + " is set:"); - Options yarnOpts = new Options(); - yarnSessionCLi.getYARNSessionCLIOptions(yarnOpts); - formatter.printHelp(" ", yarnOpts); + + // prints options from all available command-line classes + for (Map.Entry entry: customCommandLine.entrySet()) { + formatter.setSyntaxPrefix(" Additional arguments if -m " + entry.getKey() + " is set:"); + Options customOpts = new Options(); + entry.getValue().addOptions(customOpts); + formatter.printHelp(" ", customOpts); + System.out.println(); + } + System.out.println(); } @@ -376,7 +400,63 @@ public static InfoOptions parseInfoCommand(String[] args) throws CliArgsExceptio } } - public static FlinkYarnSessionCli getFlinkYarnSessionCli() { - return yarnSessionCLi; + public static Map getAllCustomCommandLine() { + if (customCommandLine.isEmpty()) { + LOG.warn("No custom command-line classes were loaded."); + } + return Collections.unmodifiableMap(customCommandLine); + } + + private static String getCliIdentifierString() { + StringBuilder builder = new StringBuilder(); + boolean first = true; + for (String identifier : customCommandLine.keySet()) { + if (!first) { + builder.append(", "); + } + first = false; + builder.append("'").append(identifier).append("'"); + } + return builder.toString(); + } + + /** + * Gets the custom command-line for this identifier. + * @param identifier The unique identifier for this command-line implementation. + * @return CustomCommandLine or null if none was found + */ + public static CustomCommandLine getActiveCustomCommandLine(String identifier) { + return CliFrontendParser.getAllCustomCommandLine().get(identifier); } + + private static void loadCustomCommandLine(String className, Object... params) { + + try { + Class customCliClass = + Class.forName(className).asSubclass(CustomCommandLine.class); + + // construct class types from the parameters + Class[] types = new Class[params.length]; + for (int i = 0; i < params.length; i++) { + Preconditions.checkNotNull(params[i], "Parameters for custom command-lines may not be null."); + types[i] = params[i].getClass(); + } + + Constructor constructor = customCliClass.getConstructor(types); + final CustomCommandLine cli = constructor.newInstance(params); + + String cliIdentifier = Preconditions.checkNotNull(cli.getIdentifier()); + CustomCommandLine existing = customCommandLine.put(cliIdentifier, cli); + + if (existing != null) { + throw new IllegalStateException("Attempted to register " + cliIdentifier + + " but there is already a command-line with this identifier."); + } + } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException | InstantiationException + | InvocationTargetException e) { + LOG.warn("Unable to locate custom CLI class {}. " + + "Flink is not compiled with support for this class.", className, e); + } + } + } diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java new file mode 100644 index 0000000000000..cd5e0e6acd082 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java @@ -0,0 +1,57 @@ +/* + * 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.cli; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.Configuration; + + +/** + * Custom command-line interface to load hooks for the command-line interface. + */ +public interface CustomCommandLine { + + /** + * Returns a unique identifier for this custom command-line. + * @return An unique identifier string + */ + String getIdentifier(); + + /** + * Adds custom options to the existing options. + * @param baseOptions The existing options. + */ + void addOptions(Options baseOptions); + + /** + * Retrieves a client for a running cluster + * @param config The Flink config + * @return Client if a cluster could be retrieve, null otherwise + */ + ClusterClient retrieveCluster(Configuration config) throws Exception; + + /** + * Creates the client for the cluster + * @param applicationName The application name to use + * @param commandLine The command-line options parsed by the CliFrontend + * @return The client to communicate with the cluster which the CustomCommandLine brought up. + */ + ClusterType createClient(String applicationName, CommandLine commandLine) throws Exception; +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java new file mode 100644 index 0000000000000..cf0595bc44d64 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java @@ -0,0 +1,41 @@ +/* + * 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.deployment; + + +import org.apache.flink.client.program.ClusterClient; + +/** + * A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication. + */ +public interface ClusterDescriptor { + + /** + * Returns a String containing details about the cluster (NodeManagers, available memory, ...) + * + */ + String getClusterDescription() throws Exception; + + /** + * Triggers deployment of a cluster + * @return Client for the cluster + * @throws Exception + */ + ClientType deploy() throws Exception; +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java similarity index 68% rename from flink-clients/src/main/java/org/apache/flink/client/program/Client.java rename to flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index dcf542ad09c71..b56428d76680c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -19,17 +19,21 @@ package org.apache.flink.client.program; import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; import java.net.URISyntaxException; import java.net.URL; import java.util.Collections; import java.util.List; import java.util.Map; +import akka.actor.ActorRef; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; -import org.apache.flink.api.common.accumulators.AccumulatorHelper; -import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.optimizer.CompilerException; import org.apache.flink.optimizer.DataStatistics; import org.apache.flink.optimizer.Optimizer; @@ -44,6 +48,7 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -51,45 +56,43 @@ import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsFound; import org.apache.flink.runtime.messages.accumulators.RequestAccumulatorResults; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.net.ConnectionUtils; import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.Some; +import scala.Tuple2; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; import akka.actor.ActorSystem; + /** * Encapsulates the functionality necessary to submit a program to a remote cluster. */ -public class Client { +public abstract class ClusterClient { - private static final Logger LOG = LoggerFactory.getLogger(Client.class); + private static final Logger LOG = LoggerFactory.getLogger(ClusterClient.class); /** The optimizer used in the optimization of batch programs */ final Optimizer compiler; /** The actor system used to communicate with the JobManager */ - private final ActorSystem actorSystem; + protected final ActorSystem actorSystem; /** Configuration of the client */ - private final Configuration config; + protected final Configuration flinkConfig; /** Timeout for futures */ - private final FiniteDuration timeout; + protected final FiniteDuration timeout; /** Lookup timeout for the job manager retrieval service */ private final FiniteDuration lookupTimeout; - /** - * If != -1, this field specifies the total number of available slots on the cluster - * connected to the client. - */ - private final int maxSlots; - /** Flag indicating whether to sysout print execution updates */ private boolean printStatusDuringExecution = true; @@ -100,6 +103,9 @@ public class Client { */ private JobID lastJobID; + /** Switch for blocking/detached job submission of the client */ + private boolean detachedJobSubmission = false; + // ------------------------------------------------------------------------ // Construction // ------------------------------------------------------------------------ @@ -109,53 +115,64 @@ public class Client { * configuration. This method will try to resolve the JobManager hostname and throw an exception * if that is not possible. * - * @param config The config used to obtain the job-manager's address, and used to configure the optimizer. + * @param flinkConfig The config used to obtain the job-manager's address, and used to configure the optimizer. * * @throws java.io.IOException Thrown, if the client's actor system could not be started. - * @throws java.net.UnknownHostException Thrown, if the JobManager's hostname could not be resolved. */ - public Client(Configuration config) throws IOException { - this(config, -1); + public ClusterClient(Configuration flinkConfig) throws IOException { + + this.flinkConfig = Preconditions.checkNotNull(flinkConfig); + this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig); + + this.timeout = AkkaUtils.getClientTimeout(flinkConfig); + this.lookupTimeout = AkkaUtils.getLookupTimeout(flinkConfig); + + this.actorSystem = createActorSystem(); } + // ------------------------------------------------------------------------ + // Startup & Shutdown + // ------------------------------------------------------------------------ + /** - * Creates a new instance of the class that submits the jobs to a job-manager. - * at the given address using the default port. - * - * @param config The configuration for the client-side processes, like the optimizer. - * @param maxSlots maxSlots The number of maxSlots on the cluster if != -1. - * - * @throws java.io.IOException Thrown, if the client's actor system could not be started. - * @throws java.net.UnknownHostException Thrown, if the JobManager's hostname could not be resolved. + * Method to create the ActorSystem of the Client. May be overriden in subclasses. + * @return ActorSystem + * @throws IOException */ - public Client(Configuration config, int maxSlots) throws IOException { - this.config = Preconditions.checkNotNull(config); - this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); - this.maxSlots = maxSlots; + protected ActorSystem createActorSystem() throws IOException { - LOG.info("Starting client actor system"); + if (actorSystem != null) { + throw new RuntimeException("This method may only be called once."); + } - try { - this.actorSystem = JobClient.startJobClientActorSystem(config); - } catch (Exception e) { - throw new IOException("Could start client actor system.", e); + // start actor system + LOG.info("Starting client actor system."); + + String hostName = flinkConfig.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); + int port = flinkConfig.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1); + if (hostName == null || port == -1) { + throw new IOException("The initial JobManager address has not been set correctly."); } + InetSocketAddress initialJobManagerAddress = new InetSocketAddress(hostName, port); - timeout = AkkaUtils.getClientTimeout(config); - lookupTimeout = AkkaUtils.getLookupTimeout(config); + // find name of own public interface, able to connect to the JM + // try to find address for 2 seconds. log after 400 ms. + InetAddress ownHostname = ConnectionUtils.findConnectingAddress(initialJobManagerAddress, 2000, 400); + return AkkaUtils.createActorSystem(flinkConfig, + new Some<>(new Tuple2(ownHostname.getCanonicalHostName(), 0))); } - // ------------------------------------------------------------------------ - // Startup & Shutdown - // ------------------------------------------------------------------------ - /** * Shuts down the client. This stops the internal actor system and actors. */ public void shutdown() { - if (!this.actorSystem.isTerminated()) { - this.actorSystem.shutdown(); - this.actorSystem.awaitTermination(); + try { + finalizeCluster(); + } finally { + if (!this.actorSystem.isTerminated()) { + this.actorSystem.shutdown(); + this.actorSystem.awaitTermination(); + } } } @@ -181,11 +198,30 @@ public boolean getPrintStatusDuringExecution() { } /** - * @return -1 if unknown. The maximum number of available processing slots at the Flink cluster - * connected to this client. + * Gets the current JobManager address from the Flink configuration (may change in case of a HA setup). + * @return The address (host and port) of the leading JobManager */ - public int getMaxSlots() { - return this.maxSlots; + public InetSocketAddress getJobManagerAddressFromConfig() { + try { + String hostName = flinkConfig.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); + int port = flinkConfig.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1); + return new InetSocketAddress(hostName, port); + } catch (Exception e) { + throw new RuntimeException("Failed to retrieve JobManager address", e); + } + } + + /** + * Gets the current JobManager address (may change in case of a HA setup). + * @return The address (host and port) of the leading JobManager + */ + public InetSocketAddress getJobManagerAddress() { + try { + final ActorRef jmActor = getJobManagerGateway().actor(); + return AkkaUtils.getInetSockeAddressFromAkkaURL(jmActor.path().toSerializationFormat()); + } catch (Exception e) { + throw new RuntimeException("Failed to retrieve JobManager address", e); + } } // ------------------------------------------------------------------------ @@ -232,50 +268,39 @@ public static OptimizedPlan getOptimizedPlan(Optimizer compiler, Plan p, int par // Program submission / execution // ------------------------------------------------------------------------ - public JobSubmissionResult runBlocking(PackagedProgram prog, int parallelism) throws ProgramInvocationException { - Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader()); - if (prog.isUsingProgramEntryPoint()) { - return runBlocking(prog.getPlanWithJars(), parallelism, prog.getSavepointPath()); - } - else if (prog.isUsingInteractiveMode()) { - LOG.info("Starting program in interactive mode"); - ContextEnvironment.setAsContext(new ContextEnvironmentFactory(this, prog.getAllLibraries(), - prog.getClasspaths(), prog.getUserCodeClassLoader(), parallelism, true, - prog.getSavepointPath())); - - // invoke here - try { - prog.invokeInteractiveModeForExecution(); - } - finally { - ContextEnvironment.unsetContext(); - } - - return new JobSubmissionResult(lastJobID); - } - else { - throw new RuntimeException(); - } - } - - public JobSubmissionResult runDetached(PackagedProgram prog, int parallelism) + /** + * General purpose method to run a user jar from the CliFrontend in either blocking or detached mode, depending + * on whether {@code setDetached(true)} or {@code setDetached(false)}. + * @param prog the packaged program + * @param parallelism the parallelism to execute the contained Flink job + * @return The result of the execution + * @throws ProgramInvocationException + */ + public JobSubmissionResult run(PackagedProgram prog, int parallelism) throws ProgramInvocationException { Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader()); if (prog.isUsingProgramEntryPoint()) { - return runDetached(prog.getPlanWithJars(), parallelism, prog.getSavepointPath()); + return run(prog.getPlanWithJars(), parallelism, prog.getSavepointPath()); } else if (prog.isUsingInteractiveMode()) { LOG.info("Starting program in interactive mode"); ContextEnvironmentFactory factory = new ContextEnvironmentFactory(this, prog.getAllLibraries(), - prog.getClasspaths(), prog.getUserCodeClassLoader(), parallelism, false, + prog.getClasspaths(), prog.getUserCodeClassLoader(), parallelism, isDetached(), prog.getSavepointPath()); ContextEnvironment.setAsContext(factory); - // invoke here try { + // invoke main method prog.invokeInteractiveModeForExecution(); - return ((DetachedEnvironment) factory.getLastEnvCreated()).finalizeExecute(); + if (isDetached()) { + // in detached mode, we execute the whole user code to extract the Flink job, afterwards we run it here + return ((DetachedEnvironment) factory.getLastEnvCreated()).finalizeExecute(); + } + else { + // in blocking mode, we execute all Flink jobs contained in the user code and then return here + return new JobSubmissionResult(lastJobID); + } } finally { ContextEnvironment.unsetContext(); @@ -286,8 +311,8 @@ else if (prog.isUsingInteractiveMode()) { } } - public JobExecutionResult runBlocking(JobWithJars program, int parallelism) throws ProgramInvocationException { - return runBlocking(program, parallelism, null); + public JobSubmissionResult run(JobWithJars program, int parallelism) throws ProgramInvocationException { + return run(program, parallelism, null); } /** @@ -304,7 +329,7 @@ public JobExecutionResult runBlocking(JobWithJars program, int parallelism) thro * i.e. the job-manager is unreachable, or due to the fact that the * parallel execution failed. */ - public JobExecutionResult runBlocking(JobWithJars program, int parallelism, String savepointPath) + public JobSubmissionResult run(JobWithJars program, int parallelism, String savepointPath) throws CompilerException, ProgramInvocationException { ClassLoader classLoader = program.getUserCodeClassLoader(); if (classLoader == null) { @@ -312,76 +337,53 @@ public JobExecutionResult runBlocking(JobWithJars program, int parallelism, Stri } OptimizedPlan optPlan = getOptimizedPlan(compiler, program, parallelism); - return runBlocking(optPlan, program.getJarFiles(), program.getClasspaths(), classLoader, savepointPath); - } - - public JobSubmissionResult runDetached(JobWithJars program, int parallelism) throws ProgramInvocationException { - return runDetached(program, parallelism, null); + return run(optPlan, program.getJarFiles(), program.getClasspaths(), classLoader, savepointPath); } - /** - * Submits a program to the Flink cluster to which this client is connected. The call returns after the - * program was submitted and does not wait for the program to complete. - * - * @param program The program to be executed. - * @param parallelism The default parallelism to use when running the program. The default parallelism is used - * when the program does not set a parallelism by itself. - * - * @throws CompilerException Thrown, if the compiler encounters an illegal situation. - * @throws ProgramInvocationException Thrown, if the program could not be instantiated from its jar file, - * or if the submission failed. That might be either due to an I/O problem, - * i.e. the job-manager is unreachable. - */ - public JobSubmissionResult runDetached(JobWithJars program, int parallelism, String savepointPath) - throws CompilerException, ProgramInvocationException { - ClassLoader classLoader = program.getUserCodeClassLoader(); - if (classLoader == null) { - throw new IllegalArgumentException("The given JobWithJars does not provide a usercode class loader."); - } - - OptimizedPlan optimizedPlan = getOptimizedPlan(compiler, program, parallelism); - return runDetached(optimizedPlan, program.getJarFiles(), program.getClasspaths(), classLoader, savepointPath); - } - - public JobExecutionResult runBlocking( + public JobSubmissionResult run( FlinkPlan compiledPlan, List libraries, List classpaths, ClassLoader classLoader) throws ProgramInvocationException { - return runBlocking(compiledPlan, libraries, classpaths, classLoader, null); - } - - public JobExecutionResult runBlocking(FlinkPlan compiledPlan, List libraries, List classpaths, - ClassLoader classLoader, String savepointPath) throws ProgramInvocationException - { - JobGraph job = getJobGraph(compiledPlan, libraries, classpaths, savepointPath); - return runBlocking(job, classLoader); - } - - public JobSubmissionResult runDetached(FlinkPlan compiledPlan, List libraries, List classpaths, ClassLoader classLoader) throws ProgramInvocationException { - return runDetached(compiledPlan, libraries, classpaths, classLoader, null); + return run(compiledPlan, libraries, classpaths, classLoader, null); } - public JobSubmissionResult runDetached(FlinkPlan compiledPlan, List libraries, List classpaths, - ClassLoader classLoader, String savepointPath) throws ProgramInvocationException + public JobSubmissionResult run(FlinkPlan compiledPlan, + List libraries, List classpaths, ClassLoader classLoader, String savepointPath) + throws ProgramInvocationException { JobGraph job = getJobGraph(compiledPlan, libraries, classpaths, savepointPath); - return runDetached(job, classLoader); + return submitJob(job, classLoader); } - public JobExecutionResult runBlocking(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { + /** + * Submits a JobGraph blocking. + * @param jobGraph The JobGraph + * @param classLoader User code class loader to deserialize the results and errors (may contain custom classes). + * @return JobExecutionResult + * @throws ProgramInvocationException + */ + public JobExecutionResult run(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { LeaderRetrievalService leaderRetrievalService; try { - leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config); + leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig); } catch (Exception e) { - throw new ProgramInvocationException("Could not create the leader retrieval service.", e); + throw new ProgramInvocationException("Could not create the leader retrieval service", e); } try { this.lastJobID = jobGraph.getJobID(); - return JobClient.submitJobAndWait(actorSystem, leaderRetrievalService, jobGraph, timeout, printStatusDuringExecution, classLoader); + return JobClient.submitJobAndWait(actorSystem, + leaderRetrievalService, jobGraph, timeout, printStatusDuringExecution, classLoader); } catch (JobExecutionException e) { throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e); } } + /** + * Submits a JobGraph detached. + * @param jobGraph The JobGraph + * @param classLoader User code class loader to deserialize the results and errors (may contain custom classes). + * @return JobSubmissionResult + * @throws ProgramInvocationException + */ public JobSubmissionResult runDetached(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { ActorGateway jobManagerGateway; @@ -391,19 +393,11 @@ public JobSubmissionResult runDetached(JobGraph jobGraph, ClassLoader classLoade throw new ProgramInvocationException("Failed to retrieve the JobManager gateway.", e); } - LOG.info("Checking and uploading JAR files"); try { - JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout); - } - catch (IOException e) { - throw new ProgramInvocationException("Could not upload the program's JAR files to the JobManager.", e); - } - try { - this.lastJobID = jobGraph.getJobID(); JobClient.submitJobDetached(jobManagerGateway, jobGraph, timeout, classLoader); return new JobSubmissionResult(jobGraph.getJobID()); } catch (JobExecutionException e) { - throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e); + throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e); } } @@ -581,7 +575,7 @@ private JobGraph getJobGraph(FlinkPlan optPlan, List jarFiles, List cl job = ((StreamingPlan) optPlan).getJobGraph(); job.setSavepointPath(savepointPath); } else { - JobGraphGenerator gen = new JobGraphGenerator(this.config); + JobGraphGenerator gen = new JobGraphGenerator(this.flinkConfig); job = gen.compileJobGraph((OptimizedPlan) optPlan); } @@ -609,16 +603,93 @@ private JobGraph getJobGraph(FlinkPlan optPlan, List jarFiles, List cl * @return ActorGateway of the current job manager leader * @throws Exception */ - private ActorGateway getJobManagerGateway() throws Exception { + protected ActorGateway getJobManagerGateway() throws Exception { LOG.info("Looking up JobManager"); - LeaderRetrievalService leaderRetrievalService; - - leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config); return LeaderRetrievalUtils.retrieveLeaderGateway( - leaderRetrievalService, + LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig), actorSystem, lookupTimeout); } + /** + * Logs and prints to sysout if printing to stdout is enabled. + * @param message The message to log/print + */ + protected void logAndSysout(String message) { + LOG.info(message); + if (printStatusDuringExecution) { + System.out.println(message); + } + } + + // ------------------------------------------------------------------------ + // Abstract methods to be implemented by the cluster specific Client + // ------------------------------------------------------------------------ + + /** + * Returns an URL (as a string) to the JobManager web interface + */ + public abstract String getWebInterfaceURL(); + + /** + * Returns the latest cluster status, with number of Taskmanagers and slots + */ + public abstract GetClusterStatusResponse getClusterStatus(); + + /** + * May return new messages from the cluster. + * Messages can be for example about failed containers or container launch requests. + */ + protected abstract List getNewMessages(); + + /** + * Returns a string representation of the cluster. + */ + protected abstract String getClusterIdentifier(); + + /** + * Request the cluster to shut down or disconnect. + */ + protected abstract void finalizeCluster(); + + /** + * Set the mode of this client (detached or blocking job execution). + * @param isDetached If true, the client will submit programs detached via the {@code run} method + */ + public void setDetached(boolean isDetached) { + this.detachedJobSubmission = isDetached; + } + + /** + * A flag to indicate whether this clients submits jobs detached. + * @return True if the Client submits detached, false otherwise + */ + public boolean isDetached() { + return detachedJobSubmission; + } + + /** + * Return the Flink configuration object + * @return The Flink configuration object + */ + public Configuration getFlinkConfiguration() { + return flinkConfig.clone(); + } + + /** + * The client may define an upper limit on the number of slots to use + * @return -1 if unknown + */ + public abstract int getMaxSlots(); + + /** + * Calls the subclasses' submitJob method. It may decide to simply call one of the run methods or it may perform + * some custom job submission logic. + * @param jobGraph The JobGraph to be submitted + * @return JobSubmissionResult + */ + protected abstract JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) + throws ProgramInvocationException; + } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java index dfb5f2e86f650..fe2d7e0261cef 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java @@ -34,7 +34,7 @@ */ public class ContextEnvironment extends ExecutionEnvironment { - protected final Client client; + protected final ClusterClient client; protected final List jarFilesToAttach; @@ -44,8 +44,8 @@ public class ContextEnvironment extends ExecutionEnvironment { protected final String savepointPath; - public ContextEnvironment(Client remoteConnection, List jarFiles, List classpaths, - ClassLoader userCodeClassLoader, String savepointPath) { + public ContextEnvironment(ClusterClient remoteConnection, List jarFiles, List classpaths, + ClassLoader userCodeClassLoader, String savepointPath) { this.client = remoteConnection; this.jarFilesToAttach = jarFiles; this.classpathsToAttach = classpaths; @@ -58,7 +58,7 @@ public JobExecutionResult execute(String jobName) throws Exception { Plan p = createProgramPlan(jobName); JobWithJars toRun = new JobWithJars(p, this.jarFilesToAttach, this.classpathsToAttach, this.userCodeClassLoader); - this.lastJobExecutionResult = client.runBlocking(toRun, getParallelism(), savepointPath); + this.lastJobExecutionResult = client.run(toRun, getParallelism(), savepointPath).getJobExecutionResult(); return this.lastJobExecutionResult; } @@ -66,7 +66,7 @@ public JobExecutionResult execute(String jobName) throws Exception { public String getExecutionPlan() throws Exception { Plan plan = createProgramPlan("unnamed job"); - OptimizedPlan op = Client.getOptimizedPlan(client.compiler, plan, getParallelism()); + OptimizedPlan op = ClusterClient.getOptimizedPlan(client.compiler, plan, getParallelism()); PlanJSONDumpGenerator gen = new PlanJSONDumpGenerator(); return gen.getOptimizerPlanAsJSON(op); } @@ -83,7 +83,7 @@ public String toString() { + ") : " + getIdString(); } - public Client getClient() { + public ClusterClient getClient() { return this.client; } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java index e820bad205cff..f9b1fc21140f6 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java @@ -32,7 +32,7 @@ */ public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory { - private final Client client; + private final ClusterClient client; private final List jarFilesToAttach; @@ -42,34 +42,34 @@ public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory { private final int defaultParallelism; - private final boolean wait; + private final boolean isDetached; private ExecutionEnvironment lastEnvCreated; private String savepointPath; - public ContextEnvironmentFactory(Client client, List jarFilesToAttach, + public ContextEnvironmentFactory(ClusterClient client, List jarFilesToAttach, List classpathsToAttach, ClassLoader userCodeClassLoader, int defaultParallelism, - boolean wait, String savepointPath) + boolean isDetached, String savepointPath) { this.client = client; this.jarFilesToAttach = jarFilesToAttach; this.classpathsToAttach = classpathsToAttach; this.userCodeClassLoader = userCodeClassLoader; this.defaultParallelism = defaultParallelism; - this.wait = wait; + this.isDetached = isDetached; this.savepointPath = savepointPath; } @Override public ExecutionEnvironment createExecutionEnvironment() { - if (!wait && lastEnvCreated != null) { + if (isDetached && lastEnvCreated != null) { throw new InvalidProgramException("Multiple enviornments cannot be created in detached mode"); } - lastEnvCreated = wait ? - new ContextEnvironment(client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath) : - new DetachedEnvironment(client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath); + lastEnvCreated = isDetached ? + new DetachedEnvironment(client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath): + new ContextEnvironment(client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath); if (defaultParallelism > 0) { lastEnvCreated.setParallelism(defaultParallelism); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java index 037c36b06bf30..8298933790e29 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/DetachedEnvironment.java @@ -42,7 +42,7 @@ public class DetachedEnvironment extends ContextEnvironment { private static final Logger LOG = LoggerFactory.getLogger(DetachedEnvironment.class); public DetachedEnvironment( - Client remoteConnection, + ClusterClient remoteConnection, List jarFiles, List classpaths, ClassLoader userCodeClassLoader, @@ -53,7 +53,7 @@ public DetachedEnvironment( @Override public JobExecutionResult execute(String jobName) throws Exception { Plan p = createProgramPlan(jobName); - setDetachedPlan(Client.getOptimizedPlan(client.compiler, p, getParallelism())); + setDetachedPlan(ClusterClient.getOptimizedPlan(client.compiler, p, getParallelism())); LOG.warn("Job was executed in detached mode, the results will be available on completion."); this.lastJobExecutionResult = DetachedJobExecutionResult.INSTANCE; return this.lastJobExecutionResult; @@ -72,7 +72,7 @@ public void setDetachedPlan(FlinkPlan plan) { * Finishes this Context Environment's execution by explicitly running the plan constructed. */ JobSubmissionResult finalizeExecute() throws ProgramInvocationException { - return client.runDetached(detachedPlan, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath); + return client.run(detachedPlan, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointPath); } public static final class DetachedJobExecutionResult extends JobExecutionResult { diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java new file mode 100644 index 0000000000000..82f350a62e1bd --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java @@ -0,0 +1,98 @@ +/* + * 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.program; + +import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.clusterframework.messages.GetClusterStatus; +import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import scala.concurrent.Await; +import scala.concurrent.Future; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * Cluster client for communication with an standalone (on-premise) cluster or an existing cluster that has been + * brought up independently of a specific job. + */ +public class StandaloneClusterClient extends ClusterClient { + + public StandaloneClusterClient(Configuration config) throws IOException { + super(config); + } + + + @Override + public String getWebInterfaceURL() { + String host = this.getJobManagerAddress().getHostName(); + int port = getFlinkConfiguration().getInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, + ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT); + return "http://" + host + ":" + port; + } + + @Override + public GetClusterStatusResponse getClusterStatus() { + ActorGateway jmGateway; + try { + jmGateway = getJobManagerGateway(); + Future future = jmGateway.ask(GetClusterStatus.getInstance(), timeout); + Object result = Await.result(future, timeout); + if (result instanceof GetClusterStatusResponse) { + return (GetClusterStatusResponse) result; + } else { + throw new RuntimeException("Received the wrong reply " + result + " from cluster."); + } + } catch (Exception e) { + throw new RuntimeException("Couldn't retrieve the Cluster status.", e); + } + } + + @Override + public List getNewMessages() { + return Collections.emptyList(); + } + + @Override + public String getClusterIdentifier() { + return "Standalone cluster with JobManager running at " + this.getJobManagerAddress(); + } + + @Override + public int getMaxSlots() { + return -1; + } + + @Override + protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) + throws ProgramInvocationException { + if (isDetached()) { + return super.runDetached(jobGraph, classLoader); + } else { + return super.run(jobGraph, classLoader); + } + } + + @Override + protected void finalizeCluster() {} + +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java index 9d0b691a10352..de85ca864b014 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java @@ -18,17 +18,13 @@ package org.apache.flink.client; -import static org.junit.Assert.assertEquals; +import static org.apache.flink.client.CliFrontendTestUtils.checkJobManagerAddress; import static org.junit.Assert.fail; import static org.mockito.Mockito.*; -import java.io.File; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; import org.apache.flink.client.cli.CommandLineOptions; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.junit.Before; import org.junit.BeforeClass; @@ -46,56 +42,17 @@ public class CliFrontendAddressConfigurationTest { @Rule public TemporaryFolder folder = new TemporaryFolder(); - + @BeforeClass public static void init() { CliFrontendTestUtils.pipeSystemOutToNull(); } - + @Before public void clearConfig() { CliFrontendTestUtils.clearGlobalConfiguration(); } - @Test - public void testInvalidConfigAndNoOption() { - try { - CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir()); - CommandLineOptions options = mock(CommandLineOptions.class); - - frontend.updateConfig(options); - Configuration config = frontend.getConfiguration(); - - checkJobManagerAddress(config, null, -1); - - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testInvalidConfigAndOption() { - try { - CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir()); - - CommandLineOptions options = mock(CommandLineOptions.class); - when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788"); - - frontend.updateConfig(options); - Configuration config = frontend.getConfiguration(); - - InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788); - - checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - @Test public void testValidConfig() { try { @@ -112,83 +69,38 @@ public void testValidConfig() { CliFrontendTestUtils.TEST_JOB_MANAGER_PORT); } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + e.printStackTrace(); + fail(e.getMessage()); + } } - /** - * Test that the CliFrontent is able to pick up the .yarn-properties file from a specified location. - */ @Test - public void testYarnConfig() { + public void testInvalidConfigAndNoOption() { try { - File tmpFolder = folder.newFolder(); - String currentUser = System.getProperty("user.name"); - - // copy reference flink-conf.yaml to temporary test directory and append custom configuration path. - File confFile = new File(CliFrontendRunTest.class.getResource("/testconfigwithyarn/flink-conf.yaml").getFile()); - File testConfFile = new File(tmpFolder, "flink-conf.yaml"); - org.apache.commons.io.FileUtils.copyFile(confFile, testConfFile); - String toAppend = "\nyarn.properties-file.location: " + tmpFolder; - // append to flink-conf.yaml - Files.write(testConfFile.toPath(), toAppend.getBytes(), StandardOpenOption.APPEND); - // copy .yarn-properties- - File propertiesFile = new File(CliFrontendRunTest.class.getResource("/testconfigwithyarn/.yarn-properties").getFile()); - File testPropertiesFile = new File(tmpFolder, ".yarn-properties-"+currentUser); - org.apache.commons.io.FileUtils.copyFile(propertiesFile, testPropertiesFile); - - // start CLI Frontend - CliFrontend frontend = new CliFrontend(tmpFolder.getAbsolutePath()); - + CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir()); CommandLineOptions options = mock(CommandLineOptions.class); frontend.updateConfig(options); Configuration config = frontend.getConfiguration(); - checkJobManagerAddress( - config, - CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_ADDRESS, - CliFrontendTestUtils.TEST_YARN_JOB_MANAGER_PORT); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testInvalidYarnConfig() { - try { - CliFrontend cli = new CliFrontend(CliFrontendTestUtils.getConfigDirWithInvalidYarnFile()); - - CommandLineOptions options = mock(CommandLineOptions.class); - - cli.updateConfig(options); - - Configuration config = cli.getConfiguration(); + checkJobManagerAddress(config, null, -1); - checkJobManagerAddress( - config, - CliFrontendTestUtils.TEST_JOB_MANAGER_ADDRESS, - CliFrontendTestUtils.TEST_JOB_MANAGER_PORT); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } } - + @Test - public void testManualOptionsOverridesConfig() { + public void testInvalidConfigAndOption() { try { - CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getInvalidConfigDir()); CommandLineOptions options = mock(CommandLineOptions.class); when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788"); frontend.updateConfig(options); - Configuration config = frontend.getConfiguration(); InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788); @@ -200,11 +112,11 @@ public void testManualOptionsOverridesConfig() { fail(e.getMessage()); } } - + @Test - public void testManualOptionsOverridesYarn() { + public void testManualOptionsOverridesConfig() { try { - CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDirWithYarnFile()); + CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); CommandLineOptions options = mock(CommandLineOptions.class); when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788"); @@ -223,11 +135,4 @@ public void testManualOptionsOverridesYarn() { } } - public void checkJobManagerAddress(Configuration config, String expectedAddress, int expectedPort) { - String jobManagerAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); - int jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1); - - assertEquals(expectedAddress, jobManagerAddress); - assertEquals(expectedPort, jobManagerPort); - } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java index 543974211d56d..f47ca69c20500 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java @@ -30,11 +30,10 @@ import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.ProgramOptions; import org.apache.flink.client.cli.RunOptions; -import org.apache.flink.client.program.Client; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.optimizer.DataStatistics; @@ -328,7 +327,7 @@ public Class loadClass(String name) throws ClassNotFoundException { Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), c); // we expect this to fail with a "ClassNotFoundException" - Client.getOptimizedPlanAsJson(compiler, prog, 666); + ClusterClient.getOptimizedPlanAsJson(compiler, prog, 666); fail("Should have failed with a ClassNotFoundException"); } catch (ProgramInvocationException e) { diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java index 56173bd793114..ceba6cb33124a 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java @@ -25,11 +25,10 @@ import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.CommandLineOptions; import org.apache.flink.client.cli.RunOptions; -import org.apache.flink.client.program.Client; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.junit.BeforeClass; import org.junit.Test; -import org.mockito.Mockito; public class CliFrontendRunTest { @@ -75,7 +74,7 @@ public void testRun() { // test detached mode { String[] parameters = {"-p", "2", "-d", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, false, true); + RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, true, true); assertEquals(0, testFrontend.run(parameters)); } @@ -96,9 +95,6 @@ public void testRun() { // test configure savepoint path { String[] parameters = {"-s", "expectedSavepointPath", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(1, false, false); - assertEquals(0, testFrontend.run(parameters)); - RunOptions options = CliFrontendParser.parseRunCommand(parameters); assertEquals("expectedSavepointPath", options.getSavepointPath()); } @@ -125,22 +121,16 @@ public RunTestingCliFrontend(int expectedParallelism, boolean logging, boolean i } @Override - protected int executeProgramDetached(PackagedProgram program, Client client, int parallelism) { - assertTrue(isDetached); - assertEquals(this.expectedParallelism, parallelism); - assertEquals(this.sysoutLogging, client.getPrintStatusDuringExecution()); - return 0; - } - - @Override - protected int executeProgramBlocking(PackagedProgram program, Client client, int parallelism) { - assertTrue(!isDetached); + protected int executeProgram(PackagedProgram program, ClusterClient client, int parallelism) { + assertEquals(isDetached, client.isDetached()); + assertEquals(sysoutLogging, client.getPrintStatusDuringExecution()); + assertEquals(expectedParallelism, parallelism); return 0; } @Override - protected Client getClient(CommandLineOptions options, String programName, int userParallelism, boolean detached) throws Exception { - return Mockito.mock(Client.class); + protected ClusterClient getClient(CommandLineOptions options, String programName) throws Exception { + return TestingClusterClientWithoutActorSystem.create(); } } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java index 7d01ab6fad782..1872133f2dae4 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java @@ -25,9 +25,12 @@ import java.lang.reflect.Field; import java.net.MalformedURLException; import java.util.Map; + +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; public class CliFrontendTestUtils { @@ -35,16 +38,11 @@ public class CliFrontendTestUtils { public static final String TEST_JAR_MAIN_CLASS = "org.apache.flink.client.testjar.WordCount"; public static final String TEST_JAR_CLASSLOADERTEST_CLASS = "org.apache.flink.client.testjar.JobWithExternalDependency"; - - + public static final String TEST_JOB_MANAGER_ADDRESS = "192.168.1.33"; - + public static final int TEST_JOB_MANAGER_PORT = 55443; - public static final String TEST_YARN_JOB_MANAGER_ADDRESS = "22.33.44.55"; - - public static final int TEST_YARN_JOB_MANAGER_PORT = 6655; - public static String getTestJarPath() throws FileNotFoundException, MalformedURLException { File f = new File("target/maven-test-jar.jar"); @@ -68,17 +66,7 @@ public static String getInvalidConfigDir() { String confFile = CliFrontendRunTest.class.getResource("/invalidtestconfig/flink-conf.yaml").getFile(); return new File(confFile).getAbsoluteFile().getParent(); } - - public static String getConfigDirWithYarnFile() { - String confFile = CliFrontendRunTest.class.getResource("/testconfigwithyarn/flink-conf.yaml").getFile(); - return new File(confFile).getAbsoluteFile().getParent(); - } - - public static String getConfigDirWithInvalidYarnFile() { - String confFile = CliFrontendRunTest.class.getResource("/testconfigwithinvalidyarn/flink-conf.yaml").getFile(); - return new File(confFile).getAbsoluteFile().getParent(); - } - + public static void pipeSystemOutToNull() { System.setOut(new PrintStream(new BlackholeOutputSteam())); System.setErr(new PrintStream(new BlackholeOutputSteam())); @@ -114,6 +102,14 @@ private static final class BlackholeOutputSteam extends java.io.OutputStream { @Override public void write(int b){} } + + public static void checkJobManagerAddress(Configuration config, String expectedAddress, int expectedPort) { + String jobManagerAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); + int jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1); + + assertEquals(expectedAddress, jobManagerAddress); + assertEquals(expectedPort, jobManagerPort); + } // -------------------------------------------------------------------------------------------- diff --git a/flink-clients/src/test/java/org/apache/flink/client/TestingClusterClientWithoutActorSystem.java b/flink-clients/src/test/java/org/apache/flink/client/TestingClusterClientWithoutActorSystem.java new file mode 100644 index 0000000000000..ab608cbecb8f5 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/TestingClusterClientWithoutActorSystem.java @@ -0,0 +1,55 @@ +/* + * 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 akka.actor.ActorSystem; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.StandaloneClusterClient; +import org.apache.flink.configuration.Configuration; +import org.mockito.Mockito; + +import java.io.IOException; + +/** + * A client to use in tests which does not instantiate an ActorSystem. + */ +public class TestingClusterClientWithoutActorSystem extends StandaloneClusterClient { + + private TestingClusterClientWithoutActorSystem() throws IOException { + super(new Configuration()); + } + + /** + * Do not instantiate the Actor System to save resources. + * @return Mocked ActorSystem + * @throws IOException + */ + @Override + protected ActorSystem createActorSystem() throws IOException { + return Mockito.mock(ActorSystem.class); + } + + public static ClusterClient create() { + try { + return new TestingClusterClientWithoutActorSystem(); + } catch (IOException e) { + throw new RuntimeException("Could not create TestingClientWithoutActorSystem.", e); + } + } + +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java index 14a1fff9658e2..4eb526981d401 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java @@ -98,7 +98,7 @@ private void testFailureBehavior(final InetSocketAddress unreachableEndpoint) { @Override public void run() { try { - new Client(config); + new StandaloneClusterClient(config); fail("This should fail with an exception since the JobManager is unreachable."); } catch (Throwable t) { diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index 4f9b3677d4160..96785f471d542 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -66,7 +66,7 @@ /** - * Simple and maybe stupid test to check the {@link Client} class. + * Simple and maybe stupid test to check the {@link ClusterClient} class. */ public class ClientTest { @@ -127,11 +127,12 @@ public void shutDownActorSystem() { @Test public void testDetachedMode() throws Exception{ jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME()); - Client out = new Client(config); + ClusterClient out = new StandaloneClusterClient(config); + out.setDetached(true); try { PackagedProgram prg = new PackagedProgram(TestExecuteTwice.class); - out.runDetached(prg, 1); + out.run(prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -141,7 +142,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestEager.class); - out.runDetached(prg, 1); + out.run(prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -151,7 +152,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetRuntime.class); - out.runDetached(prg, 1); + out.run(prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -161,7 +162,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetJobID.class); - out.runDetached(prg, 1); + out.run(prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -171,7 +172,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetAccumulator.class); - out.runDetached(prg, 1); + out.run(prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -181,7 +182,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetAllAccumulator.class); - out.runDetached(prg, 1); + out.run(prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -198,8 +199,9 @@ public void shouldSubmitToJobClient() { try { jobManagerSystem.actorOf(Props.create(SuccessReturningActor.class), JobManager.JOB_MANAGER_NAME()); - Client out = new Client(config); - JobSubmissionResult result = out.runDetached(program.getPlanWithJars(), 1); + ClusterClient out = new StandaloneClusterClient(config); + out.setDetached(true); + JobSubmissionResult result = out.run(program.getPlanWithJars(), 1); assertNotNull(result); @@ -219,10 +221,11 @@ public void shouldSubmitToJobClientFails() { try { jobManagerSystem.actorOf(Props.create(FailureReturningActor.class), JobManager.JOB_MANAGER_NAME()); - Client out = new Client(config); + ClusterClient out = new StandaloneClusterClient(config); + out.setDetached(true); try { - out.runDetached(program.getPlanWithJars(), 1); + out.run(program.getPlanWithJars(), 1); fail("This should fail with an exception"); } catch (ProgramInvocationException e) { @@ -258,7 +261,9 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(packagedProgramMock).invokeInteractiveModeForExecution(); try { - new Client(config).runBlocking(packagedProgramMock, 1); + ClusterClient client = new StandaloneClusterClient(config); + client.setDetached(true); + client.run(packagedProgramMock, 1); fail("Creating the local execution environment should not be possible"); } catch (InvalidProgramException e) { @@ -280,7 +285,7 @@ public void testGetExecutionPlan() { assertNotNull(prg.getPreviewPlan()); Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); - OptimizedPlan op = (OptimizedPlan) Client.getOptimizedPlan(optimizer, prg, 1); + OptimizedPlan op = (OptimizedPlan) ClusterClient.getOptimizedPlan(optimizer, prg, 1); assertNotNull(op); PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator(); diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java index be2caaf3b507a..c291adaac454c 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java @@ -54,7 +54,7 @@ public void testGetExecutionPlan() { config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, mockJmAddress.getPort()); Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); - OptimizedPlan op = (OptimizedPlan) Client.getOptimizedPlan(optimizer, prg, -1); + OptimizedPlan op = (OptimizedPlan) ClusterClient.getOptimizedPlan(optimizer, prg, -1); assertNotNull(op); PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator(); diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java index 254134559c96b..6ad250d3bbc2b 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java @@ -36,9 +36,10 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; -import org.apache.flink.client.program.Client; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.JobWithJars; import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; @@ -205,9 +206,9 @@ public void submitTopologyWithOpts(final String name, final String uploadedJarLo configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerHost); configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort); - final Client client; + final ClusterClient client; try { - client = new Client(configuration); + client = new StandaloneClusterClient(configuration); } catch (final IOException e) { throw new RuntimeException("Could not establish a connection to the job manager", e); } @@ -245,9 +246,9 @@ public void killTopologyWithOpts(final String name, final KillOptions options) t configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, this.jobManagerHost); configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, this.jobManagerPort); - final Client client; + final ClusterClient client; try { - client = new Client(configuration); + client = new StandaloneClusterClient(configuration); } catch (final IOException e) { throw new RuntimeException("Could not establish a connection to the job manager", e); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java index 7962fce8c86e6..bc5ae09a332cc 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java @@ -117,11 +117,14 @@ public Integer getIntCounterResult(String accumulatorName) { return (Integer) result; } + /** * Returns a dummy object for wrapping a JobSubmissionResult * @param result The SubmissionResult * @return a JobExecutionResult + * @deprecated Will be removed in future versions. */ + @Deprecated public static JobExecutionResult fromJobSubmissionResult(JobSubmissionResult result) { return new JobExecutionResult(result.getJobID(), -1, null); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java index 91a838ba3509c..4928b25be9a65 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java @@ -25,7 +25,7 @@ */ @Public public class JobSubmissionResult { - + private JobID jobID; public JobSubmissionResult(JobID jobID) { @@ -40,4 +40,26 @@ public JobSubmissionResult(JobID jobID) { public JobID getJobID() { return jobID; } + + /** + * Checks if this JobSubmissionResult is also a JobExecutionResult. + * See {@code getJobExecutionResult} to retrieve the JobExecutionResult. + * @return True if this is a JobExecutionResult, false otherwise + */ + public boolean isJobExecutionResults() { + return this instanceof JobExecutionResult; + } + + /** + * Returns the JobExecutionResult if available. + * @return The JobExecutionResult + * @throws ClassCastException if this is not a JobExecutionResult + */ + public JobExecutionResult getJobExecutionResult() { + if (isJobExecutionResults()) { + return (JobExecutionResult) this; + } else { + throw new ClassCastException("This JobSubmissionResult is not a JobExecutionResult."); + } + } } diff --git a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties index acb9d1abd8e35..2aba6af8a3a18 100644 --- a/flink-dist/src/main/flink-bin/conf/log4j-cli.properties +++ b/flink-dist/src/main/flink-bin/conf/log4j-cli.properties @@ -29,7 +29,7 @@ log4j.appender.file.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %- # Log output from org.apache.flink.yarn to the console. This is used by the # CliFrontend class when using a per-job YARN cluster. log4j.logger.org.apache.flink.yarn=INFO, console -log4j.logger.org.apache.flink.client.FlinkYarnSessionCli=INFO, console +log4j.logger.org.apache.flink.yarn.cli.FlinkYarnSessionCli=INFO, console log4j.logger.org.apache.hadoop=INFO, console log4j.appender.console=org.apache.log4j.ConsoleAppender diff --git a/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh b/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh index aa87c89e39e08..16f8ab901b73a 100755 --- a/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh +++ b/flink-dist/src/main/flink-bin/yarn-bin/yarn-session.sh @@ -52,5 +52,5 @@ log_setting="-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4 export FLINK_CONF_DIR -$JAVA_RUN $JVM_ARGS -classpath $CC_CLASSPATH:$HADOOP_CLASSPATH:$HADOOP_CONF_DIR:$YARN_CONF_DIR $log_setting org.apache.flink.client.FlinkYarnSessionCli -ship $bin/../lib/ -j $FLINK_LIB_DIR/flink-dist*.jar "$@" +$JAVA_RUN $JVM_ARGS -classpath $CC_CLASSPATH:$HADOOP_CLASSPATH:$HADOOP_CONF_DIR:$YARN_CONF_DIR $log_setting org.apache.flink.yarn.cli.FlinkYarnSessionCli -ship $bin/../lib/ -j $FLINK_LIB_DIR/flink-dist*.jar "$@" diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java index 955122f30e87b..d79768fb06eb2 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java @@ -62,7 +62,7 @@ public static void tearDownCluster() { } catch (Throwable t) { t.printStackTrace(); - fail("Cluster shutdown caused an exception: " + t.getMessage()); + fail("ClusterClient shutdown caused an exception: " + t.getMessage()); } } diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java index 61ef446d1b677..56a0a5909d39c 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java @@ -66,7 +66,7 @@ public static void tearDownCluster() { } catch (Throwable t) { t.printStackTrace(); - fail("Cluster shutdown caused an exception: " + t.getMessage()); + fail("ClusterClient shutdown caused an exception: " + t.getMessage()); } } diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java index 6cc0b6a02eb29..7458e0821a8fd 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java @@ -67,7 +67,7 @@ public static void tearDownCluster() { } catch (Throwable t) { t.printStackTrace(); - fail("Cluster shutdown caused an exception: " + t.getMessage()); + fail("ClusterClient shutdown caused an exception: " + t.getMessage()); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java index 03bae4e957a51..9da54c1e4a339 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java @@ -20,7 +20,7 @@ import com.fasterxml.jackson.core.JsonGenerator; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.client.program.Client; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; @@ -92,7 +92,7 @@ protected Tuple2 getJobGraphAndClassLoader(Map future = jobManagerGateway.ask( @@ -214,7 +223,7 @@ public static void submitJobDetached( ListeningBehaviour.DETACHED // only receive the Acknowledge for the job submission message ), timeout); - + result = Await.result(future, timeout); } catch (TimeoutException e) { @@ -225,10 +234,10 @@ public static void submitJobDetached( throw new JobExecutionException(jobGraph.getJobID(), "Failed to send job to JobManager: " + t.getMessage(), t.getCause()); } - + if (result instanceof JobManagerMessages.JobSubmitSuccess) { JobID respondedID = ((JobManagerMessages.JobSubmitSuccess) result).jobId(); - + // validate response if (!respondedID.equals(jobGraph.getJobID())) { throw new JobExecutionException(jobGraph.getJobID(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java index 9746cef1c11a4..b7bb84e2114e5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java @@ -51,4 +51,5 @@ private ApplicationStatus(int exitCode) { public int processExitCode() { return processExitCode; } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java index 1bcb19595c262..0aaf098352369 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java @@ -538,7 +538,7 @@ private void jobManagerLeaderConnected( } // ------------------------------------------------------------------------ - // Cluster Shutdown + // ClusterClient Shutdown // ------------------------------------------------------------------------ private void shutdownCluster(ApplicationStatus status, String diagnostics) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java index 5146d7b50038c..9ad5ba5c0a30b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/messages/GetClusterStatusResponse.java @@ -21,7 +21,7 @@ import java.io.Serializable; /** - * The reply to a {@code GetClusterStatus} message sent by the resource manager. Sends over the + * The reply to a {@code GetClusterStatus} message sent by the job manager. Sends over the * current number of task managers and the available task slots. */ public class GetClusterStatusResponse implements Serializable { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java deleted file mode 100644 index c1498c56890d8..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnClient.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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.runtime.yarn; - -import org.apache.flink.configuration.Configuration; -import org.apache.hadoop.fs.Path; -import java.io.File; -import java.util.List; - -/** - * Abstract interface for an implementation of a Flink on YARN client to deploy. - * - * The Client describes the properties of the YARN application to create. - */ -public abstract class AbstractFlinkYarnClient { - - // ---- Setter for YARN Cluster properties ----- // - - /** - * @param memoryMB The amount of memory for the JobManager (in MB) - */ - public abstract void setJobManagerMemory(int memoryMB); - - /** - * @param memoryMB The memory per TaskManager (in MB) - */ - public abstract void setTaskManagerMemory(int memoryMB); - - /** - * Flink configuration - */ - public abstract void setFlinkConfiguration(org.apache.flink.configuration.Configuration conf); - - public abstract Configuration getFlinkConfiguration(); - - /** - * - * @param slots The number of TaskManager slots per TaskManager. - */ - public abstract void setTaskManagerSlots(int slots); - - /** - * @return the number of TaskManager processing slots per TaskManager. - */ - public abstract int getTaskManagerSlots(); - - /** - * @param queue Name of the YARN queue - */ - public abstract void setQueue(String queue); - - /** - * - * @param localJarPath Local Path to the Flink uberjar - */ - public abstract void setLocalJarPath(Path localJarPath); - - /** - * - * @param confPath local path to the Flink configuration file - */ - public abstract void setConfigurationFilePath(Path confPath); - - /** - * - * @param logConfPath local path to the flink logging configuration - */ - public abstract void setFlinkLoggingConfigurationPath(Path logConfPath); - public abstract Path getFlinkLoggingConfigurationPath(); - - /** - * - * @param tmCount number of TaskManagers to start - */ - public abstract void setTaskManagerCount(int tmCount); - public abstract int getTaskManagerCount(); - - /** - * @param confDirPath Path to config directory. - */ - public abstract void setConfigurationDirectory(String confDirPath); - - /** - * List of files to transfer to the YARN containers. - */ - public abstract void setShipFiles(List shipFiles); - - /** - * - * @param dynamicPropertiesEncoded Encoded String of the dynamic properties (-D configuration values of the Flink configuration) - */ - public abstract void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded); - public abstract String getDynamicPropertiesEncoded(); - - // --------------------------------------- Operations on the YARN cluster ----- // - - /** - * Returns a String containing details about the cluster (NodeManagers, available memory, ...) - * - */ - public abstract String getClusterDescription() throws Exception; - - /** - * Trigger the deployment to YARN. - * - */ - public abstract AbstractFlinkYarnCluster deploy() throws Exception; - - /** - * @param detachedMode If true, the Flink YARN client is non-blocking. That means it returns - * once Flink has been started successfully on YARN. - */ - public abstract void setDetachedMode(boolean detachedMode); - - public abstract boolean isDetached(); - - /** - * @return The string representation of the Path to the YARN session files. This is a temporary - * directory in HDFS that contains the jar files and configuration which is shipped to all the containers. - */ - public abstract String getSessionFilesDir(); - - /** - * Set a name for the YARN application - * @param name - */ - public abstract void setName(String name); -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java deleted file mode 100644 index af015c7f33d90..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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.runtime.yarn; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.List; - -/** - * Abstract class for interacting with a running Flink cluster within YARN. - */ -public abstract class AbstractFlinkYarnCluster { - - /** - * Get hostname and port of the JobManager. - */ - public abstract InetSocketAddress getJobManagerAddress(); - - /** - * Returns an URL (as a string) to the JobManager web interface, running next to the - * ApplicationMaster and JobManager in a YARN container - */ - public abstract String getWebInterfaceURL(); - - /** - * Request the YARN cluster to shut down. - * - * @param failApplication If true, the application will be marked as failed in YARN - */ - public abstract void shutdown(boolean failApplication); - - /** - * Boolean indicating whether the cluster has been stopped already - */ - public abstract boolean hasBeenStopped(); - - /** - * Returns the latest cluster status, with number of Taskmanagers and slots - */ - public abstract GetClusterStatusResponse getClusterStatus(); - - /** - * Boolean indicating whether the Flink YARN cluster is in an erronous state. - */ - public abstract boolean hasFailed(); - - /** - * @return Diagnostics if the Cluster is in "failed" state. - */ - public abstract String getDiagnostics(); - - /** - * May return new messages from the cluster. - * Messages can be for example about failed containers or container launch requests. - */ - public abstract List getNewMessages(); - - /** - * Returns a string representation of the ApplicationID assigned by YARN. - */ - public abstract String getApplicationId(); - - /** - * Flink's YARN cluster abstraction has two modes for connecting to the YARN AM. - * In the detached mode, the AM is launched and the Flink YARN client is disconnecting - * afterwards. - * In the non-detached mode, it maintains a connection with the AM to control the cluster. - * @return boolean indicating whether the cluster is a detached cluster - */ - public abstract boolean isDetached(); - - /** - * Connect the FlinkYarnCluster to the ApplicationMaster. - * - * Detached YARN sessions don't need to connect to the ApplicationMaster. - * Detached per job YARN sessions need to connect until the required number of TaskManagers have been started. - * - * @throws IOException - */ - public abstract void connectToCluster() throws IOException; - - /** - * Disconnect from the ApplicationMaster without stopping the session - * (therefore, use the {@link AbstractFlinkYarnCluster#shutdown(boolean)} method. - * - * @see AbstractFlinkYarnCluster#shutdown(boolean) - */ - public abstract void disconnect(); - - /** - * Tells the ApplicationMaster to monitor the status of JobId and stop itself once the specified - * job has finished. - * - * @param jobID Id of the job - */ - public abstract void stopAfterJob(JobID jobID); - - /** - * Return the Flink configuration object - * @return The Flink configuration object - */ - public abstract Configuration getFlinkConfiguration(); -} diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala index 7b80206064d4f..1a8870b797596 100644 --- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala +++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala @@ -19,14 +19,14 @@ package org.apache.flink.api.scala import java.io._ -import java.util.Properties -import org.apache.flink.client.{CliFrontend, ClientUtils, FlinkYarnSessionCli} -import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} +import org.apache.flink.client.cli.CliFrontendParser +import org.apache.flink.client.program.ClusterClient +import org.apache.flink.client.CliFrontend +import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration} import org.apache.flink.runtime.minicluster.{FlinkMiniCluster, LocalFlinkMiniCluster} -import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster -import org.apache.hadoop.fs.Path +import scala.collection.mutable.ArrayBuffer import scala.tools.nsc.Settings import scala.tools.nsc.interpreter._ @@ -138,7 +138,7 @@ object FlinkShell { def fetchConnectionInfo( config: Config - ): (String, Int, Option[Either[FlinkMiniCluster, AbstractFlinkYarnCluster]]) = { + ): (String, Int, Option[Either[FlinkMiniCluster, ClusterClient]]) = { config.executionMode match { case ExecutionMode.LOCAL => // Local mode val config = GlobalConfiguration.getConfiguration() @@ -217,7 +217,7 @@ object FlinkShell { repl.closeInterpreter() cluster match { case Some(Left(miniCluster)) => miniCluster.stop() - case Some(Right(yarnCluster)) => yarnCluster.shutdown(false) + case Some(Right(yarnCluster)) => yarnCluster.shutdown() case _ => } } @@ -226,71 +226,49 @@ object FlinkShell { } def deployNewYarnCluster(yarnConfig: YarnConfig) = { - val yarnClient = FlinkYarnSessionCli.getFlinkYarnClient - - // use flink-dist.jar for scala shell - val jarPath = new Path("file://" + - s"${yarnClient.getClass.getProtectionDomain.getCodeSource.getLocation.getPath}") - yarnClient.setLocalJarPath(jarPath) - - val confDirPath = CliFrontend.getConfigurationDirectoryFromEnv - val flinkConfiguration = GlobalConfiguration.getConfiguration - val confFile = new File(confDirPath + File.separator + "flink-conf.yaml") - val confPath = new Path(confFile.getAbsolutePath) - yarnClient.setFlinkConfiguration(flinkConfiguration) - yarnClient.setConfigurationDirectory(confDirPath) - yarnClient.setConfigurationFilePath(confPath) + + val args = ArrayBuffer[String]( + "-m", "yarn-cluster" + ) // number of task managers is required. yarnConfig.containers match { - case Some(containers) => yarnClient.setTaskManagerCount(containers) + case Some(containers) => args ++= Seq("-yn", containers.toString) case None => throw new IllegalArgumentException("Number of taskmanagers must be specified.") } // set configuration from user input - yarnConfig.jobManagerMemory.foreach(yarnClient.setJobManagerMemory) - yarnConfig.name.foreach(yarnClient.setName) - yarnConfig.queue.foreach(yarnClient.setQueue) - yarnConfig.slots.foreach(yarnClient.setTaskManagerSlots) - yarnConfig.taskManagerMemory.foreach(yarnClient.setTaskManagerMemory) - - // deploy - val cluster = yarnClient.deploy() + yarnConfig.jobManagerMemory.foreach((jmMem) => args ++= Seq("-yjm", jmMem.toString)) + yarnConfig.slots.foreach((tmMem) => args ++= Seq("-ytm", tmMem.toString)) + yarnConfig.name.foreach((name) => args ++= Seq("-ynm", name.toString)) + yarnConfig.queue.foreach((queue) => args ++= Seq("-yqu", queue.toString)) + yarnConfig.slots.foreach((slots) => args ++= Seq("-ys", slots.toString)) + + val customCLI = CliFrontendParser.getAllCustomCommandLine.get("yarn-cluster") + + val options = CliFrontendParser.parseRunCommand(args.toArray) + + val cluster = customCLI.createClient("Flink Scala Shell", options.getCommandLine) + val address = cluster.getJobManagerAddress.getAddress.getHostAddress val port = cluster.getJobManagerAddress.getPort - cluster.connectToCluster() (address, port, Some(Right(cluster))) } def fetchDeployedYarnClusterInfo() = { + // load configuration val globalConfig = GlobalConfiguration.getConfiguration - val propertiesLocation = CliFrontend.getYarnPropertiesLocation(globalConfig) - val propertiesFile = new File(propertiesLocation) - - // read properties - val properties = if (propertiesFile.exists()) { - println("Found YARN properties file " + propertiesFile.getAbsolutePath) - val properties = new Properties() - val inputStream = new FileInputStream(propertiesFile) - - try { - properties.load(inputStream) - } finally { - inputStream.close() - } - properties - } else { - throw new IllegalArgumentException("Scala Shell cannot fetch YARN properties.") - } + val customCLI = CliFrontendParser.getAllCustomCommandLine.get("yarn-cluster") + + val cluster = customCLI.retrieveCluster(globalConfig) - val addressInStr = properties.getProperty(CliFrontend.YARN_PROPERTIES_JOBMANAGER_KEY) - val address = ClientUtils.parseHostPortAddress(addressInStr) + val jobManager = cluster.getJobManagerAddress - (address.getHostString, address.getPort, None) + (jobManager.getHostString, jobManager.getPort, None) } def ensureYarnConfig(config: Config) = config.yarnConfig match { diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala index 4475bc84dc73e..f03cb840ac6ef 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala @@ -775,7 +775,7 @@ object ExecutionEnvironment { * configuration parameters for the Client only; Program parallelism can be set via * [[ExecutionEnvironment.setParallelism]]. * - * Cluster configuration has to be done in the remotely running Flink instance. + * ClusterClient configuration has to be done in the remotely running Flink instance. * * @param host The host name or address of the master (JobManager), where the program should be * executed. diff --git a/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java b/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java index f37969d7c0693..bc9bedcdc5d32 100644 --- a/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java +++ b/flink-streaming-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java @@ -230,4 +230,4 @@ public void process(Tuple2 element, RuntimeContext ctx, Request indexer.add(createIndexRequest(element)); } } -} \ No newline at end of file +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 03945a07700c3..333f9c08c73a5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -30,9 +30,10 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.client.program.Client; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.JobWithJars; import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -195,9 +196,9 @@ protected JobExecutionResult executeRemotely(StreamGraph streamGraph, List configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, host); configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port); - Client client; + ClusterClient client; try { - client = new Client(configuration); + client = new StandaloneClusterClient(configuration); client.setPrintStatusDuringExecution(getConfig().isSysoutLoggingEnabled()); } catch (Exception e) { @@ -205,7 +206,7 @@ protected JobExecutionResult executeRemotely(StreamGraph streamGraph, List } try { - return client.runBlocking(streamGraph, jarFiles, globalClasspaths, usercodeClassLoader); + return client.run(streamGraph, jarFiles, globalClasspaths, usercodeClassLoader).getJobExecutionResult(); } catch (ProgramInvocationException e) { throw e; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java index 59991437ce94b..0332684ecff2e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java @@ -63,7 +63,10 @@ public JobExecutionResult execute(String jobName) throws Exception { ((DetachedEnvironment) ctx).setDetachedPlan(streamGraph); return DetachedEnvironment.DetachedJobExecutionResult.INSTANCE; } else { - return ctx.getClient().runBlocking(streamGraph, ctx.getJars(), ctx.getClasspaths(), ctx.getUserCodeClassLoader(), ctx.getSavepointPath()); + return ctx + .getClient() + .run(streamGraph, ctx.getJars(), ctx.getClasspaths(), ctx.getUserCodeClassLoader(), ctx.getSavepointPath()) + .getJobExecutionResult(); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java index b1768f094b982..34a7eedaa9d3e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java @@ -78,7 +78,7 @@ public static void tearDownCluster() { } catch (Throwable t) { t.printStackTrace(); - fail("Cluster shutdown caused an exception: " + t.getMessage()); + fail("ClusterClient shutdown caused an exception: " + t.getMessage()); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java index 7dccb7dbcdf7f..09b5e7eb68677 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java @@ -69,7 +69,7 @@ public static void teardownCluster() { catch (Throwable t) { System.err.println("Error stopping cluster on shutdown"); t.printStackTrace(); - fail("Cluster shutdown caused an exception: " + t.getMessage()); + fail("ClusterClient shutdown caused an exception: " + t.getMessage()); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java index 4437db1c522d2..0a0f451753764 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCase.java @@ -69,7 +69,7 @@ public static void teardownCluster() { catch (Throwable t) { System.err.println("Error stopping cluster on shutdown"); t.printStackTrace(); - fail("Cluster shutdown caused an exception: " + t.getMessage()); + fail("ClusterClient shutdown caused an exception: " + t.getMessage()); } } diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index 239a85a935878..e8492113409bb 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -92,11 +92,16 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - - - 1 - ../ - + + + integration-tests + + + 1 + ../ + + + diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java new file mode 100644 index 0000000000000..c6a1adeb57ab4 --- /dev/null +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java @@ -0,0 +1,220 @@ +/* + * 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.client.CliFrontend; +import org.apache.flink.client.cli.CommandLineOptions; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.junit.*; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.OutputStream; +import java.io.PrintStream; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests that verify that the CLI client picks up the correct address for the JobManager + * from configuration and configs. + */ +public class CliFrontendYarnAddressConfigurationTest { + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private final static PrintStream OUT = System.out; + private final static PrintStream ERR = System.err; + + @BeforeClass + public static void disableStdOutErr() { + class NullPrint extends OutputStream { + @Override + public void write(int b) {} + } + + PrintStream nullPrinter = new PrintStream(new NullPrint()); + System.setOut(nullPrinter); + System.setErr(nullPrinter); + } + + @AfterClass + public static void restoreAfterwards() { + System.setOut(OUT); + System.setErr(ERR); + } + + @Before + public void clearConfig() throws NoSuchFieldException, IllegalAccessException { + // reset GlobalConfiguration between tests + Field instance = GlobalConfiguration.class.getDeclaredField("SINGLETON"); + instance.setAccessible(true); + instance.set(null, null); + } + + private static final String TEST_YARN_JOB_MANAGER_ADDRESS = "22.33.44.55"; + private static final int TEST_YARN_JOB_MANAGER_PORT = 6655; + + private static final String propertiesFile = + "jobManager=" + TEST_YARN_JOB_MANAGER_ADDRESS + ":" + TEST_YARN_JOB_MANAGER_PORT; + + + private static final String TEST_JOB_MANAGER_ADDRESS = "192.168.1.33"; + private static final int TEST_JOB_MANAGER_PORT = 55443; + + private static final String flinkConf = + "jobmanager.rpc.address: " + TEST_JOB_MANAGER_ADDRESS + "\n" + + "jobmanager.rpc.port: " + TEST_JOB_MANAGER_PORT; + + + private static final String invalidPropertiesFile = + "jasfobManager=" + TEST_YARN_JOB_MANAGER_ADDRESS + ":asf" + TEST_YARN_JOB_MANAGER_PORT; + + + /** + * Test that the CliFrontend is able to pick up the .yarn-properties file from a specified location. + */ + @Test + public void testYarnConfig() { + try { + File tmpFolder = temporaryFolder.newFolder(); + String currentUser = System.getProperty("user.name"); + + // copy .yarn-properties- + File testPropertiesFile = new File(tmpFolder, ".yarn-properties-"+currentUser); + Files.write(testPropertiesFile.toPath(), propertiesFile.getBytes(), StandardOpenOption.CREATE); + + // copy reference flink-conf.yaml to temporary test directory and append custom configuration path. + String confFile = flinkConf + "\nyarn.properties-file.location: " + tmpFolder; + File testConfFile = new File(tmpFolder.getAbsolutePath(), "flink-conf.yaml"); + Files.write(testConfFile.toPath(), confFile.getBytes(), StandardOpenOption.CREATE); + + // start CLI Frontend + TestCLI frontend = new TestCLI(tmpFolder.getAbsolutePath()); + + CommandLineOptions options = mock(CommandLineOptions.class); + + frontend.getClient(options, "Program name"); + + frontend.updateConfig(options); + Configuration config = frontend.getConfiguration(); + + checkJobManagerAddress( + config, + TEST_YARN_JOB_MANAGER_ADDRESS, + TEST_YARN_JOB_MANAGER_PORT); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + public static class TestCLI extends CliFrontend { + TestCLI(String configDir) throws Exception { + super(configDir); + } + + @Override + public ClusterClient getClient(CommandLineOptions options, String programName) throws Exception { + return super.getClient(options, programName); + } + + @Override + public void updateConfig(CommandLineOptions options) { + super.updateConfig(options); + } + } + + @Test + public void testInvalidYarnConfig() { + try { + File tmpFolder = temporaryFolder.newFolder(); + + // copy invalid .yarn-properties- + File testPropertiesFile = new File(tmpFolder, ".yarn-properties"); + Files.write(testPropertiesFile.toPath(), invalidPropertiesFile.getBytes(), StandardOpenOption.CREATE); + + // copy reference flink-conf.yaml to temporary test directory and append custom configuration path. + String confFile = flinkConf + "\nyarn.properties-file.location: " + tmpFolder; + File testConfFile = new File(tmpFolder.getAbsolutePath(), "flink-conf.yaml"); + Files.write(testConfFile.toPath(), confFile.getBytes(), StandardOpenOption.CREATE); + + TestCLI cli = new TestCLI(tmpFolder.getAbsolutePath()); + + CommandLineOptions options = mock(CommandLineOptions.class); + + cli.updateConfig(options); + + Configuration config = cli.getConfiguration(); + + checkJobManagerAddress( + config, + TEST_JOB_MANAGER_ADDRESS, + TEST_JOB_MANAGER_PORT); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + + @Test + public void testManualOptionsOverridesYarn() { + try { + File emptyFolder = temporaryFolder.newFolder(); + TestCLI frontend = new TestCLI(emptyFolder.getAbsolutePath()); + + CommandLineOptions options = mock(CommandLineOptions.class); + when(options.getJobManagerAddress()).thenReturn("10.221.130.22:7788"); + + frontend.updateConfig(options); + + Configuration config = frontend.getConfiguration(); + + InetSocketAddress expectedAddress = new InetSocketAddress("10.221.130.22", 7788); + + checkJobManagerAddress(config, expectedAddress.getHostName(), expectedAddress.getPort()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + + private static void checkJobManagerAddress(Configuration config, String expectedAddress, int expectedPort) { + String jobManagerAddress = config.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); + int jobManagerPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, -1); + + assertEquals(expectedAddress, jobManagerAddress); + assertEquals(expectedPort, jobManagerPort); + } + +} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index 7197b64de4d81..c842bdc54969a 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -24,8 +24,7 @@ import org.apache.commons.cli.PosixParser; import org.apache.flink.client.CliFrontend; -import org.apache.flink.client.FlinkYarnSessionCli; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; +import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.test.util.TestBaseUtils; import org.junit.Assert; @@ -38,6 +37,8 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties; + public class FlinkYarnSessionCliTest { @Rule @@ -54,7 +55,7 @@ public void testDynamicProperties() throws IOException { TestBaseUtils.setEnv(map); Options options = new Options(); FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", "", false); - cli.getYARNSessionCLIOptions(options); + cli.addOptions(options); CommandLineParser parser = new PosixParser(); CommandLine cmd = null; @@ -65,11 +66,12 @@ public void testDynamicProperties() throws IOException { Assert.fail("Parsing failed with " + e.getMessage()); } - AbstractFlinkYarnClient flinkYarnClient = cli.createFlinkYarnClient(cmd); + YarnClusterDescriptor flinkYarnDescriptor = cli.createDescriptor(null, cmd); - Assert.assertNotNull(flinkYarnClient); + Assert.assertNotNull(flinkYarnDescriptor); - Map dynProperties = CliFrontend.getDynamicProperties(flinkYarnClient.getDynamicPropertiesEncoded()); + Map dynProperties = + FlinkYarnSessionCli.getDynamicProperties(flinkYarnDescriptor.getDynamicPropertiesEncoded()); Assert.assertEquals(1, dynProperties.size()); Assert.assertEquals("5 min", dynProperties.get("akka.ask.timeout")); } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingFlinkYarnClient.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java similarity index 95% rename from flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingFlinkYarnClient.java rename to flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java index dbfbfe2ae7927..386f48f9a10c3 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingFlinkYarnClient.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java @@ -30,9 +30,9 @@ * flink-yarn-tests-XXX-tests.jar and the flink-runtime-XXX-tests.jar to the set of files which * are shipped to the yarn cluster. This is necessary to load the testing classes. */ -public class TestingFlinkYarnClient extends FlinkYarnClientBase { +public class TestingYarnClusterDescriptor extends AbstractYarnClusterDescriptor { - public TestingFlinkYarnClient() { + public TestingYarnClusterDescriptor() { List filesToShip = new ArrayList<>(); File testingJar = YarnTestBase.findFile("..", new TestJarFinder("flink-yarn-tests")); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index a93abf06bd75c..4d45f169d4ed5 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -22,6 +22,7 @@ import akka.actor.PoisonPill; import akka.testkit.JavaTestKit; import org.apache.curator.test.TestingServer; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; @@ -33,7 +34,6 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.junit.AfterClass; @@ -97,7 +97,7 @@ public static void teardown() throws IOException { public void testMultipleAMKill() throws Exception { final int numberKillingAttempts = numberApplicationAttempts - 1; - TestingFlinkYarnClient flinkYarnClient = new TestingFlinkYarnClient(); + TestingYarnClusterDescriptor flinkYarnClient = new TestingYarnClusterDescriptor(); Assert.assertNotNull("unable to get yarn client", flinkYarnClient); flinkYarnClient.setTaskManagerCount(1); @@ -119,13 +119,12 @@ public void testMultipleAMKill() throws Exception { "@@" + ConfigConstants.ZOOKEEPER_RECOVERY_PATH + "=" + fsStateHandlePath + "/recovery"); flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml")); - AbstractFlinkYarnCluster yarnCluster = null; + ClusterClient yarnCluster = null; final FiniteDuration timeout = new FiniteDuration(2, TimeUnit.MINUTES); try { yarnCluster = flinkYarnClient.deploy(); - yarnCluster.connectToCluster(); final Configuration config = yarnCluster.getFlinkConfiguration(); new JavaTestKit(actorSystem) {{ @@ -169,7 +168,7 @@ protected void run() { }}; } finally { if (yarnCluster != null) { - yarnCluster.shutdown(false); + yarnCluster.shutdown(); } } } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index 38e17a5d20fa7..826a0863cb224 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -328,7 +328,7 @@ public void testTaskManagerFailure() { @Test public void testNonexistingQueue() { LOG.info("Starting testNonexistingQueue()"); - addTestAppender(FlinkYarnClient.class, Level.WARN); + addTestAppender(YarnClusterDescriptor.class, Level.WARN); runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(), "-n", "1", @@ -432,7 +432,9 @@ private void testDetachedPerJobYarnClusterInternal(String job) { "-yD", "yarn.heap-cutoff-ratio=0.5", // test if the cutoff is passed correctly "-ytm", "1024", "-ys", "2", // test requesting slots from YARN. - "--yarndetached", job, "--input", tmpInFile.getAbsoluteFile().toString(), "--output", tmpOutFolder.getAbsoluteFile().toString()}, + "--yarndetached", job, + "--input", tmpInFile.getAbsoluteFile().toString(), + "--output", tmpOutFolder.getAbsoluteFile().toString()}, "Job has been submitted with JobID", RunTypes.CLI_FRONTEND); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index cb402a3f55291..fe5400acce100 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -18,11 +18,10 @@ package org.apache.flink.yarn; -import org.apache.flink.client.FlinkYarnSessionCli; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -95,8 +94,6 @@ public void testDetachedMode() { checkForLogString("The Flink YARN client has been started in detached mode"); - Assert.assertFalse("The runner should detach.", runner.isAlive()); - LOG.info("Waiting until two containers are running"); // wait until two containers are running while(getRunningContainers() < 2) { @@ -171,7 +168,7 @@ public void testNonexistingQueue() { @Ignore("The test is too resource consuming (8.5 GB of memory)") @Test public void testResourceComputation() { - addTestAppender(FlinkYarnClient.class, Level.WARN); + addTestAppender(YarnClusterDescriptor.class, Level.WARN); LOG.info("Starting testResourceComputation()"); runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(), "-n", "5", @@ -199,7 +196,7 @@ public void testResourceComputation() { @Ignore("The test is too resource consuming (8 GB of memory)") @Test public void testfullAlloc() { - addTestAppender(FlinkYarnClient.class, Level.WARN); + addTestAppender(YarnClusterDescriptor.class, Level.WARN); LOG.info("Starting testfullAlloc()"); runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(), "-n", "2", @@ -218,7 +215,7 @@ public void testJavaAPI() { final int WAIT_TIME = 15; LOG.info("Starting testJavaAPI()"); - AbstractFlinkYarnClient flinkYarnClient = FlinkYarnSessionCli.getFlinkYarnClient(); + AbstractYarnClusterDescriptor flinkYarnClient = new YarnClusterDescriptor(); Assert.assertNotNull("unable to get yarn client", flinkYarnClient); flinkYarnClient.setTaskManagerCount(1); flinkYarnClient.setJobManagerMemory(768); @@ -231,10 +228,9 @@ public void testJavaAPI() { flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml")); // deploy - AbstractFlinkYarnCluster yarnCluster = null; + ClusterClient yarnCluster = null; try { yarnCluster = flinkYarnClient.deploy(); - yarnCluster.connectToCluster(); } catch (Exception e) { LOG.warn("Failing test", e); Assert.fail("Error while deploying YARN cluster: "+e.getMessage()); @@ -248,7 +244,7 @@ public void testJavaAPI() { } GetClusterStatusResponse status = yarnCluster.getClusterStatus(); if(status != null && status.equals(expectedStatus)) { - LOG.info("Cluster reached status " + status); + LOG.info("ClusterClient reached status " + status); break; // all good, cluster started } if(second > WAIT_TIME) { @@ -263,7 +259,7 @@ public void testJavaAPI() { LOG.info("Shutting down cluster. All tests passed"); // shutdown cluster - yarnCluster.shutdown(false); + yarnCluster.shutdown(); LOG.info("Finished testJavaAPI()"); } } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 03ab647201b7d..4de964abcdecc 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -20,7 +20,7 @@ import org.apache.commons.io.FileUtils; import org.apache.flink.client.CliFrontend; -import org.apache.flink.client.FlinkYarnSessionCli; +import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.util.TestLogger; import org.apache.hadoop.conf.Configuration; @@ -368,7 +368,7 @@ public static void startYARNWithConfig(Configuration conf) { File yarnConfFile = writeYarnSiteConfigXML(conf); map.put("YARN_CONF_DIR", yarnConfFile.getParentFile().getAbsolutePath()); - map.put("IN_TESTS", "yes we are in tests"); // see FlinkYarnClient() for more infos + map.put("IN_TESTS", "yes we are in tests"); // see YarnClusterDescriptor() for more infos TestBaseUtils.setEnv(map); Assert.assertTrue(yarnCluster.getServiceState() == Service.STATE.STARTED); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClientBase.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java similarity index 87% rename from flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClientBase.java rename to flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 6f81d09d46595..7220a29eae171 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClientBase.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -19,12 +19,11 @@ package org.apache.flink.yarn; import org.apache.flink.client.CliFrontend; -import org.apache.flink.client.FlinkYarnSessionCli; +import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.jobmanager.RecoveryMode; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -68,6 +67,10 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties; + /** * All classes in this package contain code taken from * https://github.com/apache/hadoop-common/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java?source=cc @@ -81,8 +84,10 @@ * by YARN into their local fs. * */ -public abstract class FlinkYarnClientBase extends AbstractFlinkYarnClient { - private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnClient.class); +public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor { + private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class); + + private static final String CONFIG_FILE_NAME = "flink-conf.yaml"; /** * Minimum memory requirements, checked by the Client. @@ -90,10 +95,7 @@ public abstract class FlinkYarnClientBase extends AbstractFlinkYarnClient { private static final int MIN_JM_MEMORY = 768; // the minimum memory should be higher than the min heap cutoff private static final int MIN_TM_MEMORY = 768; - private Configuration conf; - private YarnClient yarnClient; - private YarnClientApplication yarnApplication; - private Thread deploymentFailureHook = new DeploymentFailureHook(); + private Configuration conf = new YarnConfiguration(); /** * Files (usually in a distributed file system) used for the YARN session of Flink. @@ -131,15 +133,7 @@ public abstract class FlinkYarnClientBase extends AbstractFlinkYarnClient { private String customName = null; - public FlinkYarnClientBase() { - conf = new YarnConfiguration(); - if(this.yarnClient == null) { - // Create yarnClient - yarnClient = YarnClient.createYarnClient(); - yarnClient.init(conf); - yarnClient.start(); - } - + public AbstractYarnClusterDescriptor() { // for unit tests only if(System.getenv("IN_TESTS") != null) { try { @@ -148,6 +142,36 @@ public FlinkYarnClientBase() { throw new RuntimeException("Error",t); } } + + // load the config + this.configurationDirectory = CliFrontend.getConfigurationDirectoryFromEnv(); + GlobalConfiguration.loadConfiguration(configurationDirectory); + this.flinkConfiguration = GlobalConfiguration.getConfiguration(); + + File confFile = new File(configurationDirectory + File.separator + CONFIG_FILE_NAME); + if (!confFile.exists()) { + throw new RuntimeException("Unable to locate configuration file in " + confFile); + } + flinkConfigurationPath = new Path(confFile.getAbsolutePath()); + + //check if there is a logback or log4j file + if (configurationDirectory.length() > 0) { + File logback = new File(configurationDirectory + File.pathSeparator + CONFIG_FILE_LOGBACK_NAME); + if (logback.exists()) { + shipFiles.add(logback); + flinkLoggingConfigurationPath = new Path(logback.toURI()); + } + File log4j = new File(configurationDirectory + File.pathSeparator + CONFIG_FILE_LOG4J_NAME); + if (log4j.exists()) { + shipFiles.add(log4j); + if (flinkLoggingConfigurationPath != null) { + // this means there is already a logback configuration file --> fail + LOG.warn("The configuration directory ('" + configurationDirectory + "') contains both LOG4J and " + + "Logback configuration files. Please delete or rename one of them."); + } + flinkLoggingConfigurationPath = new Path(log4j.toURI()); + } + } } /** @@ -155,7 +179,6 @@ public FlinkYarnClientBase() { */ protected abstract Class getApplicationMasterClass(); - @Override public void setJobManagerMemory(int memoryMb) { if(memoryMb < MIN_JM_MEMORY) { throw new IllegalArgumentException("The JobManager memory (" + memoryMb + ") is below the minimum required memory amount " @@ -164,7 +187,6 @@ public void setJobManagerMemory(int memoryMb) { this.jobManagerMemoryMb = memoryMb; } - @Override public void setTaskManagerMemory(int memoryMb) { if(memoryMb < MIN_TM_MEMORY) { throw new IllegalArgumentException("The TaskManager memory (" + memoryMb + ") is below the minimum required memory amount " @@ -173,17 +195,14 @@ public void setTaskManagerMemory(int memoryMb) { this.taskManagerMemoryMb = memoryMb; } - @Override public void setFlinkConfiguration(org.apache.flink.configuration.Configuration conf) { this.flinkConfiguration = conf; } - @Override public org.apache.flink.configuration.Configuration getFlinkConfiguration() { return flinkConfiguration; } - @Override public void setTaskManagerSlots(int slots) { if(slots <= 0) { throw new IllegalArgumentException("Number of TaskManager slots must be positive"); @@ -191,17 +210,14 @@ public void setTaskManagerSlots(int slots) { this.slots = slots; } - @Override public int getTaskManagerSlots() { return this.slots; } - @Override public void setQueue(String queue) { this.yarnQueue = queue; } - @Override public void setLocalJarPath(Path localJarPath) { if(!localJarPath.toString().endsWith("jar")) { throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); @@ -209,27 +225,22 @@ public void setLocalJarPath(Path localJarPath) { this.flinkJarPath = localJarPath; } - @Override public void setConfigurationFilePath(Path confPath) { flinkConfigurationPath = confPath; } - @Override public void setConfigurationDirectory(String configurationDirectory) { this.configurationDirectory = configurationDirectory; } - @Override public void setFlinkLoggingConfigurationPath(Path logConfPath) { flinkLoggingConfigurationPath = logConfPath; } - @Override public Path getFlinkLoggingConfigurationPath() { return flinkLoggingConfigurationPath; } - @Override public void setTaskManagerCount(int tmCount) { if(tmCount < 1) { throw new IllegalArgumentException("The TaskManager count has to be at least 1."); @@ -237,12 +248,10 @@ public void setTaskManagerCount(int tmCount) { this.taskManagerCount = tmCount; } - @Override public int getTaskManagerCount() { return this.taskManagerCount; } - @Override public void setShipFiles(List shipFiles) { for(File shipFile: shipFiles) { // remove uberjar from ship list (by default everything in the lib/ folder is added to @@ -253,18 +262,16 @@ public void setShipFiles(List shipFiles) { } } - @Override public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { this.dynamicPropertiesEncoded = dynamicPropertiesEncoded; } - @Override public String getDynamicPropertiesEncoded() { return this.dynamicPropertiesEncoded; } - public void isReadyForDeployment() throws YarnDeploymentException { + private void isReadyForDeployment() throws YarnDeploymentException { if(taskManagerCount <= 0) { throw new YarnDeploymentException("Taskmanager count must be positive"); } @@ -290,7 +297,7 @@ public void isReadyForDeployment() throws YarnDeploymentException { } } - public static boolean allocateResource(int[] nodeManagers, int toAllocate) { + private static boolean allocateResource(int[] nodeManagers, int toAllocate) { for(int i = 0; i < nodeManagers.length; i++) { if(nodeManagers[i] >= toAllocate) { nodeManagers[i] -= toAllocate; @@ -300,18 +307,28 @@ public static boolean allocateResource(int[] nodeManagers, int toAllocate) { return false; } - @Override public void setDetachedMode(boolean detachedMode) { this.detached = detachedMode; } - @Override - public boolean isDetached() { + public boolean isDetachedMode() { return detached; } + + /** + * Gets a Hadoop Yarn client + * @return Returns a YarnClient which has to be shutdown manually + */ + public static YarnClient getYarnClient(Configuration conf) { + YarnClient yarnClient = YarnClient.createYarnClient(); + yarnClient.init(conf); + yarnClient.start(); + return yarnClient; + } + @Override - public AbstractFlinkYarnCluster deploy() throws Exception { + public YarnClusterClient deploy() throws Exception { UserGroupInformation.setConfiguration(conf); UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); @@ -321,9 +338,9 @@ public AbstractFlinkYarnCluster deploy() throws Exception { throw new YarnDeploymentException("In secure mode. Please provide Kerberos credentials in order to authenticate. " + "You may use kinit to authenticate and request a TGT from the Kerberos server."); } - return ugi.doAs(new PrivilegedExceptionAction() { + return ugi.doAs(new PrivilegedExceptionAction() { @Override - public AbstractFlinkYarnCluster run() throws Exception { + public YarnClusterClient run() throws Exception { return deployInternal(); } }); @@ -332,13 +349,11 @@ public AbstractFlinkYarnCluster run() throws Exception { } } - - /** * This method will block until the ApplicationMaster/JobManager have been * deployed on YARN. */ - protected AbstractFlinkYarnCluster deployInternal() throws Exception { + protected YarnClusterClient deployInternal() throws Exception { isReadyForDeployment(); LOG.info("Using values:"); @@ -347,23 +362,26 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { LOG.info("\tTaskManager memory = {}", taskManagerMemoryMb); // Create application via yarnClient - yarnApplication = yarnClient.createApplication(); + final YarnClient yarnClient = getYarnClient(conf); + final YarnClientApplication yarnApplication = yarnClient.createApplication(); GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse(); // ------------------ Add dynamic properties to local flinkConfiguraton ------ - Map dynProperties = CliFrontend.getDynamicProperties(dynamicPropertiesEncoded); + Map dynProperties = getDynamicProperties(dynamicPropertiesEncoded); for (Map.Entry dynProperty : dynProperties.entrySet()) { flinkConfiguration.setString(dynProperty.getKey(), dynProperty.getValue()); } + // ------------------ Set default file system scheme ------------------------- + try { org.apache.flink.core.fs.FileSystem.setDefaultScheme(flinkConfiguration); } catch (IOException e) { throw new IOException("Error while setting the default " + "filesystem scheme from configuration.", e); } - // ------------------ Check if the specified queue exists -------------- + // ------------------ Check if the specified queue exists -------------------- try { List queues = yarnClient.getAllQueues(); @@ -393,7 +411,7 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { } } - // ------------------ Check if the YARN Cluster has the requested resources -------------- + // ------------------ Check if the YARN ClusterClient has the requested resources -------------- // the yarnMinAllocationMB specifies the smallest possible container allocation size. // all allocations below this value are automatically set to this value. @@ -416,13 +434,13 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { Resource maxRes = appResponse.getMaximumResourceCapability(); final String NOTE = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n"; if(jobManagerMemoryMb > maxRes.getMemory() ) { - failSessionDuringDeployment(); + failSessionDuringDeployment(yarnClient, yarnApplication); throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n" + "Maximum Memory: " + maxRes.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + NOTE); } if(taskManagerMemoryMb > maxRes.getMemory() ) { - failSessionDuringDeployment(); + failSessionDuringDeployment(yarnClient, yarnApplication); throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n" + "Maximum Memory: " + maxRes.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + NOTE); } @@ -472,9 +490,9 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { // respect custom JVM options in the YAML file final String javaOpts = flinkConfiguration.getString(ConfigConstants.FLINK_JVM_OPTIONS, ""); - String logbackFile = configurationDirectory + File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME; + String logbackFile = configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME; boolean hasLogback = new File(logbackFile).exists(); - String log4jFile = configurationDirectory + File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; + String log4jFile = configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME; boolean hasLog4j = new File(log4jFile).exists(); if(hasLogback) { @@ -495,11 +513,11 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { amCommand += " -Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\""; if(hasLogback) { - amCommand += " -Dlogback.configurationFile=file:" + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME; + amCommand += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME; } if(hasLog4j) { - amCommand += " -Dlog4j.configuration=file:" + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; + amCommand += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME; } } @@ -632,14 +650,15 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { } // add a hook to clean up in case deployment fails + Thread deploymentFailureHook = new DeploymentFailureHook(yarnClient, yarnApplication); Runtime.getRuntime().addShutdownHook(deploymentFailureHook); LOG.info("Submitting application master " + appId); yarnClient.submitApplication(appContext); LOG.info("Waiting for the cluster to be allocated"); int waittime = 0; + ApplicationReport report; loop: while( true ) { - ApplicationReport report; try { report = yarnClient.getApplicationReport(appId); } catch (IOException e) { @@ -670,7 +689,7 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { Thread.sleep(1000); } // print the application id for user to cancel themselves. - if (isDetached()) { + if (isDetachedMode()) { LOG.info("The Flink YARN client has been started in detached mode. In order to stop " + "Flink on YARN, use the following command or a YARN web interface to stop " + "it:\nyarn application -kill " + appId + "\nPlease also note that the " + @@ -682,8 +701,17 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { } catch (IllegalStateException e) { // we're already in the shut down hook. } + + String host = report.getHost(); + int port = report.getRpcPort(); + String trackingURL = report.getTrackingUrl(); + + // Correctly initialize the Flink config + flinkConfiguration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, host); + flinkConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port); + // the Flink cluster is deployed in YARN. Represent cluster - return new FlinkYarnCluster(yarnClient, appId, conf, flinkConfiguration, sessionFilesDir, detached); + return new YarnClusterClient(this, yarnClient, report, flinkConfiguration, sessionFilesDir); } /** @@ -691,7 +719,7 @@ protected AbstractFlinkYarnCluster deployInternal() throws Exception { * * Use this method to kill the App before it has been properly deployed */ - private void failSessionDuringDeployment() { + private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { LOG.info("Killing YARN application"); try { @@ -742,9 +770,10 @@ public String getClusterDescription() throws Exception { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintStream ps = new PrintStream(baos); + YarnClient yarnClient = getYarnClient(conf); YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); - ps.append("NodeManagers in the Cluster " + metrics.getNumNodeManagers()); + ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers()); List nodes = yarnClient.getNodeReports(NodeState.RUNNING); final String format = "|%-16s |%-16s %n"; ps.printf("|Property |Value %n"); @@ -772,12 +801,10 @@ public String getClusterDescription() throws Exception { return baos.toString(); } - @Override public String getSessionFilesDir() { return sessionFilesDir.toString(); } - @Override public void setName(String name) { if(name == null) { throw new IllegalArgumentException("The passed name is null"); @@ -873,7 +900,7 @@ public void setAttemptFailuresValidityInterval( } } - public static class YarnDeploymentException extends RuntimeException { + private static class YarnDeploymentException extends RuntimeException { private static final long serialVersionUID = -812040641215388943L; public YarnDeploymentException() { @@ -889,10 +916,19 @@ public YarnDeploymentException(String message, Throwable cause) { } private class DeploymentFailureHook extends Thread { + + DeploymentFailureHook(YarnClient yarnClient, YarnClientApplication yarnApplication) { + this.yarnClient = yarnClient; + this.yarnApplication = yarnApplication; + } + + private YarnClient yarnClient; + private YarnClientApplication yarnApplication; + @Override public void run() { LOG.info("Cancelling deployment from Deployment Failure Hook"); - failSessionDuringDeployment(); + failSessionDuringDeployment(yarnClient, yarnApplication); LOG.info("Deleting files in " + sessionFilesDir); try { FileSystem fs = FileSystem.get(conf); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index 1d0afc4f081ee..24b5a35949883 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -22,7 +22,6 @@ import akka.actor.ActorSystem; import akka.actor.Props; -import org.apache.flink.client.CliFrontend; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; @@ -39,6 +38,7 @@ import org.apache.flink.runtime.util.SignalHandler; import org.apache.flink.runtime.webmonitor.WebMonitor; +import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.security.Credentials; @@ -192,7 +192,7 @@ protected int runApplicationMaster() { // Flink configuration final Map dynamicProperties = - CliFrontend.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES)); + FlinkYarnSessionCli.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES)); LOG.debug("YARN dynamic properties: {}", dynamicProperties); final Configuration config = createConfiguration(currDir, dynamicProperties); @@ -292,8 +292,7 @@ protected int runApplicationMaster() { // 3: Flink's Yarn ResourceManager LOG.debug("Starting YARN Flink Resource Manager"); - // we need the leader retrieval service here to be informed of new - // leader session IDs, even though there can be only one leader ever + // we need the leader retrieval service here to be informed of new leaders and session IDs LeaderRetrievalService leaderRetriever = LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java similarity index 68% rename from flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java rename to flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java index 3b4822893bb25..a5b8af767cba5 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java @@ -18,7 +18,6 @@ package org.apache.flink.yarn; import akka.actor.ActorRef; -import akka.actor.ActorSystem; import static akka.pattern.Patterns.ask; @@ -27,14 +26,16 @@ import akka.pattern.Patterns; import akka.util.Timeout; import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; import org.apache.flink.runtime.clusterframework.messages.InfoMessage; -import org.apache.flink.runtime.net.ConnectionUtils; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -42,7 +43,6 @@ import org.apache.hadoop.service.Service; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -51,14 +51,11 @@ import scala.None$; import scala.Option; import scala.Some; -import scala.Tuple2; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; @@ -66,8 +63,9 @@ /** * Java representation of a running Flink cluster within YARN. */ -public class FlinkYarnCluster extends AbstractFlinkYarnCluster { - private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnCluster.class); +public class YarnClusterClient extends ClusterClient { + + private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClient.class); private static final int POLLING_THREAD_INTERVAL_MS = 1000; @@ -78,19 +76,19 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster { private final Configuration hadoopConfig; // (HDFS) location of the files required to run on YARN. Needed here to delete them on shutdown. private final Path sessionFilesDir; - private final InetSocketAddress jobManagerAddress; + + /** The leader retrieval service for connecting to the cluster and finding the active leader. */ + private final LeaderRetrievalService leaderRetrievalService; //---------- Class internal fields ------------------- - private ActorSystem actorSystem; - private ActorRef applicationClient; - private ApplicationReport intialAppReport; + private final AbstractYarnClusterDescriptor clusterDescriptor; + private final ActorRef applicationClient; private final FiniteDuration akkaDuration; private final Timeout akkaTimeout; - private final ApplicationId applicationId; - private final boolean detached; - private final org.apache.flink.configuration.Configuration flinkConfig; + private final ApplicationReport applicationId; private final ApplicationId appId; + private final String trackingURL; private boolean isConnected = false; @@ -98,65 +96,32 @@ public class FlinkYarnCluster extends AbstractFlinkYarnCluster { /** * Create a new Flink on YARN cluster. * + * @param clusterDescriptor The descriptor used at cluster creation * @param yarnClient Client to talk to YARN - * @param appId the YARN application ID - * @param hadoopConfig Hadoop configuration + * @param appReport the YARN application ID * @param flinkConfig Flink configuration * @param sessionFilesDir Location of files required for YARN session - * @param detached Set to true if no actor system or RPC communication with the cluster should be established * @throws IOException * @throws YarnException */ - public FlinkYarnCluster( - final YarnClient yarnClient, - final ApplicationId appId, - Configuration hadoopConfig, - org.apache.flink.configuration.Configuration flinkConfig, - Path sessionFilesDir, - boolean detached) throws IOException, YarnException { + public YarnClusterClient( + final AbstractYarnClusterDescriptor clusterDescriptor, + final YarnClient yarnClient, + final ApplicationReport appReport, + org.apache.flink.configuration.Configuration flinkConfig, + Path sessionFilesDir) throws IOException, YarnException { + + super(flinkConfig); + this.akkaDuration = AkkaUtils.getTimeout(flinkConfig); this.akkaTimeout = Timeout.durationToTimeout(akkaDuration); + this.clusterDescriptor = clusterDescriptor; this.yarnClient = yarnClient; - this.hadoopConfig = hadoopConfig; + this.hadoopConfig = yarnClient.getConfig(); this.sessionFilesDir = sessionFilesDir; - this.applicationId = appId; - this.detached = detached; - this.flinkConfig = flinkConfig; - this.appId = appId; - - // get one application report manually - intialAppReport = yarnClient.getApplicationReport(appId); - String jobManagerHost = intialAppReport.getHost(); - int jobManagerPort = intialAppReport.getRpcPort(); - this.jobManagerAddress = new InetSocketAddress(jobManagerHost, jobManagerPort); - } - - /** - * Connect the FlinkYarnCluster to the ApplicationMaster. - * - * Detached YARN sessions don't need to connect to the ApplicationMaster. - * Detached per job YARN sessions need to connect until the required number of TaskManagers have been started. - * - * @throws IOException - */ - public void connectToCluster() throws IOException { - if(isConnected) { - throw new IllegalStateException("Can not connect to the cluster again"); - } - - // start actor system - LOG.info("Start actor system."); - // find name of own public interface, able to connect to the JM - // try to find address for 2 seconds. log after 400 ms. - InetAddress ownHostname = ConnectionUtils.findConnectingAddress(jobManagerAddress, 2000, 400); - actorSystem = AkkaUtils.createActorSystem(flinkConfig, - new Some>(new Tuple2(ownHostname.getCanonicalHostName(), 0))); - - // Create the leader election service - flinkConfig.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, this.jobManagerAddress.getHostName()); - flinkConfig.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, this.jobManagerAddress.getPort()); - - LeaderRetrievalService leaderRetrievalService; + this.applicationId = appReport; + this.appId = appReport.getApplicationId(); + this.trackingURL = appReport.getTrackingUrl(); try { leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig); @@ -164,6 +129,11 @@ public void connectToCluster() throws IOException { throw new IOException("Could not create the leader retrieval service.", e); } + + if (isConnected) { + throw new IllegalStateException("Already connected to the cluster."); + } + // start application client LOG.info("Start application client."); @@ -185,16 +155,16 @@ public void run() { ApplicationReport appReport = yarnClient.getApplicationReport(appId); LOG.info("Application " + appId + " finished with state " + appReport - .getYarnApplicationState() + " and final state " + appReport - .getFinalApplicationStatus() + " at " + appReport.getFinishTime()); + .getYarnApplicationState() + " and final state " + appReport + .getFinalApplicationStatus() + " at " + appReport.getFinishTime()); if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED || appReport.getYarnApplicationState() - == YarnApplicationState.KILLED) { + == YarnApplicationState.KILLED) { LOG.warn("Application failed. Diagnostics " + appReport.getDiagnostics()); LOG.warn("If log aggregation is activated in the Hadoop cluster, we recommend to retrieve " - + "the full application log using this command:\n" - + "\tyarn logs -applicationId " + appReport.getApplicationId() + "\n" - + "(It sometimes takes a few seconds until the logs are aggregated)"); + + "the full application log using this command:\n" + + "\tyarn logs -applicationId " + appReport.getApplicationId() + "\n" + + "(It sometimes takes a few seconds until the logs are aggregated)"); } } catch (Exception e) { LOG.warn("Error while getting final application report", e); @@ -211,14 +181,38 @@ public void run() { Runtime.getRuntime().addShutdownHook(clientShutdownHook); isConnected = true; + + logAndSysout("Waiting until all TaskManagers have connected"); + + while(true) { + GetClusterStatusResponse status = getClusterStatus(); + if (status != null) { + if (status.numRegisteredTaskManagers() < clusterDescriptor.getTaskManagerCount()) { + logAndSysout("TaskManager status (" + status.numRegisteredTaskManagers() + "/" + + clusterDescriptor.getTaskManagerCount() + ")"); + } else { + logAndSysout("All TaskManagers are connected"); + break; + } + } else { + logAndSysout("No status updates from the YARN cluster received so far. Waiting ..."); + } + + try { + Thread.sleep(500); + } catch (InterruptedException e) { + LOG.error("Interrupted while waiting for TaskManagers"); + System.err.println("Thread is interrupted"); + throw new IOException("Interrupted while waiting for TaskManagers", e); + } + } } - @Override public void disconnect() { if(!isConnected) { throw new IllegalStateException("Can not disconnect from an unconnected cluster."); } - LOG.info("Disconnecting FlinkYarnCluster from ApplicationMaster"); + LOG.info("Disconnecting YarnClusterClient from ApplicationMaster"); if(!Runtime.getRuntime().removeShutdownHook(clientShutdownHook)) { LOG.warn("Error while removing the shutdown hook. The YARN session might be killed unintentionally"); @@ -246,10 +240,9 @@ public void disconnect() { // -------------------------- Interaction with the cluster ------------------------ /* - * This call blocks until the message has been recevied. + * Tells the Cluster to monitor the status of JobId and stop itself once the specified job has finished. */ - @Override - public void stopAfterJob(JobID jobID) { + private void stopAfterJob(JobID jobID) { Preconditions.checkNotNull(jobID, "The job id must not be null"); Future messageReceived = ask(applicationClient, new YarnMessages.LocalStopAMAfterJob(jobID), akkaTimeout); try { @@ -265,28 +258,35 @@ public org.apache.flink.configuration.Configuration getFlinkConfiguration() { } @Override - public InetSocketAddress getJobManagerAddress() { - return jobManagerAddress; + public int getMaxSlots() { + int maxSlots = clusterDescriptor.getTaskManagerCount() * clusterDescriptor.getTaskManagerSlots(); + return maxSlots > 0 ? maxSlots : -1; } @Override - public String getWebInterfaceURL() { - String url = this.intialAppReport.getTrackingUrl(); - // there seems to be a difference between HD 2.2.0 and 2.6.0 - if(!url.startsWith("http://")) { - url = "http://" + url; + protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { + if (isDetached()) { + JobSubmissionResult result = super.runDetached(jobGraph, classLoader); + stopAfterJob(jobGraph.getJobID()); + return result; + } else { + return super.run(jobGraph, classLoader); } - return url; } @Override - public String getApplicationId() { - return applicationId.toString(); + public String getWebInterfaceURL() { + // there seems to be a difference between HD 2.2.0 and 2.6.0 + if(!trackingURL.startsWith("http://")) { + return "http://" + trackingURL; + } else { + return trackingURL; + } } @Override - public boolean isDetached() { - return this.detached; + public String getClusterIdentifier() { + return applicationId.getApplicationId().toString(); } /** @@ -297,15 +297,15 @@ public GetClusterStatusResponse getClusterStatus() { if(!isConnected) { throw new IllegalStateException("The cluster is not connected to the ApplicationMaster."); } - if(hasBeenStopped()) { - throw new RuntimeException("The FlinkYarnCluster has already been stopped"); + if(hasBeenShutdown()) { + throw new RuntimeException("The YarnClusterClient has already been stopped"); } Future clusterStatusOption = ask(applicationClient, YarnMessages.getLocalGetyarnClusterStatus(), akkaTimeout); Object clusterStatus; try { clusterStatus = Await.result(clusterStatusOption, akkaDuration); } catch (Exception e) { - throw new RuntimeException("Unable to get Cluster status from Application Client", e); + throw new RuntimeException("Unable to get ClusterClient status from Application Client", e); } if(clusterStatus instanceof None$) { return null; @@ -316,25 +316,25 @@ public GetClusterStatusResponse getClusterStatus() { } } - @Override - public boolean hasFailed() { + public ApplicationStatus getApplicationStatus() { if(!isConnected) { throw new IllegalStateException("The cluster has been connected to the ApplicationMaster."); } if(pollingRunner == null) { - LOG.warn("FlinkYarnCluster.hasFailed() has been called on an uninitialized cluster." + + LOG.warn("YarnClusterClient.getApplicationStatus() has been called on an uninitialized cluster." + "The system might be in an erroneous state"); } ApplicationReport lastReport = pollingRunner.getLastReport(); if(lastReport == null) { - LOG.warn("FlinkYarnCluster.hasFailed() has been called on a cluster that didn't receive a status so far." + + LOG.warn("YarnClusterClient.getApplicationStatus() has been called on a cluster that didn't receive a status so far." + "The system might be in an erroneous state"); - return false; + return ApplicationStatus.UNKNOWN; } else { YarnApplicationState appState = lastReport.getYarnApplicationState(); - boolean status = (appState == YarnApplicationState.FAILED || - appState == YarnApplicationState.KILLED); - if(status) { + ApplicationStatus status = + (appState == YarnApplicationState.FAILED || appState == YarnApplicationState.KILLED) ? + ApplicationStatus.FAILED : ApplicationStatus.SUCCEEDED; + if(status != ApplicationStatus.SUCCEEDED) { LOG.warn("YARN reported application state {}", appState); LOG.warn("Diagnostics: {}", lastReport.getDiagnostics()); } @@ -343,13 +343,12 @@ public boolean hasFailed() { } - @Override - public String getDiagnostics() { + private String getDiagnostics() { if(!isConnected) { throw new IllegalStateException("The cluster has been connected to the ApplicationMaster."); } - if (!hasFailed()) { + if (getApplicationStatus() == ApplicationStatus.SUCCEEDED) { LOG.warn("getDiagnostics() called for cluster which is not in failed state"); } ApplicationReport lastReport = pollingRunner.getLastReport(); @@ -367,8 +366,8 @@ public List getNewMessages() { throw new IllegalStateException("The cluster has been connected to the ApplicationMaster."); } - if(hasBeenStopped()) { - throw new RuntimeException("The FlinkYarnCluster has already been stopped"); + if(hasBeenShutdown()) { + throw new RuntimeException("The YarnClusterClient has already been stopped"); } List ret = new ArrayList(); @@ -413,14 +412,36 @@ public List getNewMessages() { private AtomicBoolean hasBeenShutDown = new AtomicBoolean(false); /** - * Shutdown the YARN cluster. - * @param failApplication whether we should fail the YARN application (in case of errors in Flink) + * Shuts down or disconnects from the YARN cluster. */ @Override - public void shutdown(boolean failApplication) { - if(!isConnected) { - throw new IllegalStateException("The cluster has been connected to the ApplicationMaster."); + public void finalizeCluster() { + + if (!isConnected) { + throw new IllegalStateException("The cluster has been not been connected to the ApplicationMaster."); + } + + if (isDetached()) { + // only disconnect if we are running detached + disconnect(); + return; + } + + // show cluster status + + List msgs = getNewMessages(); + if (msgs != null && msgs.size() > 1) { + + logAndSysout("The following messages were created by the YARN cluster while running the Job:"); + for (String msg : msgs) { + logAndSysout(msg); + } } + if (getApplicationStatus() != ApplicationStatus.SUCCEEDED) { + logAndSysout("YARN cluster is in non-successful state " + getApplicationStatus()); + logAndSysout("YARN Diagnostics: " + getDiagnostics()); + } + if(hasBeenShutDown.getAndSet(true)) { return; @@ -436,14 +457,8 @@ public void shutdown(boolean failApplication) { LOG.info("Sending shutdown request to the Application Master"); if(applicationClient != ActorRef.noSender()) { try { - FinalApplicationStatus finalStatus; - if (failApplication) { - finalStatus = FinalApplicationStatus.FAILED; - } else { - finalStatus = FinalApplicationStatus.SUCCEEDED; - } Future response = Patterns.ask(applicationClient, - new YarnMessages.LocalStopYarnSession(finalStatus, + new YarnMessages.LocalStopYarnSession(getApplicationStatus(), "Flink YARN Client requested shutdown"), new Timeout(akkaDuration)); Await.ready(response, akkaDuration); @@ -454,11 +469,9 @@ public void shutdown(boolean failApplication) { actorSystem.shutdown(); actorSystem.awaitTermination(); - - actorSystem = null; } - LOG.info("Deleting files in " + sessionFilesDir ); + LOG.info("Deleting files in " + sessionFilesDir); try { FileSystem shutFS = FileSystem.get(hadoopConfig); shutFS.delete(sessionFilesDir, true); // delete conf and jar file. @@ -486,23 +499,22 @@ public void shutdown(boolean failApplication) { yarnClient = null; // set null to clearly see if somebody wants to access it afterwards. } - @Override - public boolean hasBeenStopped() { + public boolean hasBeenShutdown() { return hasBeenShutDown.get(); } - public class ClientShutdownHook extends Thread { + private class ClientShutdownHook extends Thread { @Override public void run() { - LOG.info("Shutting down FlinkYarnCluster from the client shutdown hook"); - shutdown(false); + LOG.info("Shutting down YarnClusterClient from the client shutdown hook"); + shutdown(); } } // -------------------------- Polling ------------------------ - public static class PollingThread extends Thread { + private static class PollingThread extends Thread { AtomicBoolean running = new AtomicBoolean(true); private YarnClient yarnClient; @@ -543,10 +555,11 @@ public void run() { LOG.warn("Error while getting application report", e); } try { - Thread.sleep(FlinkYarnCluster.POLLING_THREAD_INTERVAL_MS); + Thread.sleep(YarnClusterClient.POLLING_THREAD_INTERVAL_MS); } catch (InterruptedException e) { LOG.error("Polling thread got interrupted", e); Thread.currentThread().interrupt(); // pass interrupt. + stopRunner(); } } if(running.get() && !yarnClient.isInState(Service.STATE.STARTED)) { @@ -556,4 +569,9 @@ public void run() { } } + @Override + public boolean isDetached() { + // either we have set detached mode using the general '-d' flag or using the Yarn CLI flag 'yd' + return super.isDetached() || clusterDescriptor.isDetachedMode(); + } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java similarity index 83% rename from flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java rename to flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 467e06d5e1833..43e7c7be09be7 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnClient.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -18,9 +18,9 @@ package org.apache.flink.yarn; /** - * Default implementation of {@link FlinkYarnClientBase} which starts an {@link YarnApplicationMasterRunner}. + * Default implementation of {@link AbstractYarnClusterDescriptor} which starts an {@link YarnApplicationMasterRunner}. */ -public class FlinkYarnClient extends FlinkYarnClientBase { +public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor { @Override protected Class getApplicationMasterClass() { return YarnApplicationMasterRunner.class; diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java similarity index 58% rename from flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java rename to flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index bb61ffb7b6d8b..a2375c532d546 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.yarn.cli; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -24,42 +24,63 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.client.CliFrontend; +import org.apache.flink.client.ClientUtils; +import org.apache.flink.client.cli.CliFrontendParser; +import org.apache.flink.client.cli.CustomCommandLine; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.StandaloneClusterClient; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.flink.yarn.YarnClusterClient; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient; -import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; -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.FileInputStream; import java.io.FileOutputStream; import java.io.FilenameFilter; import java.io.IOException; +import java.io.InputStream; import java.io.InputStreamReader; import java.io.OutputStream; +import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Properties; /** * Class handling the command line interface to the YARN session. */ -public class FlinkYarnSessionCli { +public class FlinkYarnSessionCli implements CustomCommandLine { 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; + /** The id for the CommandLine interface */ + private static final String ID = "yarn-cluster"; + + // YARN-session related constants + private static final String YARN_PROPERTIES_FILE = ".yarn-properties-"; + private static final String YARN_PROPERTIES_JOBMANAGER_KEY = "jobManager"; + private static final String YARN_PROPERTIES_PARALLELISM = "parallelism"; + private static final String YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING = "dynamicPropertiesString"; + + private static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; // this has to be a regex for String.split() //------------------------------------ Command Line argument options ------------------------- // the prefix transformation is used by the CliFrontend static constructor. @@ -85,11 +106,12 @@ public class FlinkYarnSessionCli { private final boolean acceptInteractiveInput; //------------------------------------ Internal fields ------------------------- - private AbstractFlinkYarnCluster yarnCluster = null; + private YarnClusterClient yarnCluster = null; private boolean detachedMode = false; - /** Default yarn application name. */ - private String defaultApplicationName = null; + public FlinkYarnSessionCli(String shortPrefix, String longPrefix) { + this(shortPrefix, longPrefix, true); + } public FlinkYarnSessionCli(String shortPrefix, String longPrefix, boolean acceptInteractiveInput) { this.acceptInteractiveInput = acceptInteractiveInput; @@ -109,23 +131,81 @@ public FlinkYarnSessionCli(String shortPrefix, String longPrefix, boolean accept } /** - * Creates a new Yarn Client. - * @param cmd the command line to parse options from - * @return an instance of the client or null if there was an error + * Resumes from a Flink Yarn properties file + * @param flinkConfiguration The flink configuration + * @return True if the properties were loaded, false otherwise */ - public AbstractFlinkYarnClient createFlinkYarnClient(CommandLine cmd) { + private boolean resumeFromYarnProperties(Configuration flinkConfiguration) { + // load the YARN properties + File propertiesFile = new File(getYarnPropertiesLocation(flinkConfiguration)); + if (!propertiesFile.exists()) { + return false; + } + + logAndSysout("Found YARN properties file " + propertiesFile.getAbsolutePath()); + + Properties yarnProperties = new Properties(); + try { + try (InputStream is = new FileInputStream(propertiesFile)) { + yarnProperties.load(is); + } + } + catch (IOException e) { + throw new RuntimeException("Cannot read the YARN properties file", e); + } + + // configure the default parallelism from YARN + String propParallelism = yarnProperties.getProperty(YARN_PROPERTIES_PARALLELISM); + if (propParallelism != null) { // maybe the property is not set + try { + int parallelism = Integer.parseInt(propParallelism); + flinkConfiguration.setInteger(ConfigConstants.DEFAULT_PARALLELISM_KEY, parallelism); + + logAndSysout("YARN properties set default parallelism to " + parallelism); + } + catch (NumberFormatException e) { + throw new RuntimeException("Error while parsing the YARN properties: " + + "Property " + YARN_PROPERTIES_PARALLELISM + " is not an integer."); + } + } + + // get the JobManager address from the YARN properties + String address = yarnProperties.getProperty(YARN_PROPERTIES_JOBMANAGER_KEY); + InetSocketAddress jobManagerAddress; + if (address != null) { + try { + jobManagerAddress = ClientUtils.parseHostPortAddress(address); + // store address in config from where it is retrieved by the retrieval service + CliFrontend.writeJobManagerAddressToConfig(flinkConfiguration, jobManagerAddress); + } + catch (Exception e) { + throw new RuntimeException("YARN properties contain an invalid entry for JobManager address.", e); + } + + logAndSysout("Using JobManager address from YARN properties " + jobManagerAddress); + } - AbstractFlinkYarnClient flinkYarnClient = getFlinkYarnClient(); - if (flinkYarnClient == null) { - return null; + // handle the YARN client's dynamic properties + String dynamicPropertiesEncoded = yarnProperties.getProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING); + Map dynamicProperties = getDynamicProperties(dynamicPropertiesEncoded); + for (Map.Entry dynamicProperty : dynamicProperties.entrySet()) { + flinkConfiguration.setString(dynamicProperty.getKey(), dynamicProperty.getValue()); } + return true; + } + + public YarnClusterDescriptor createDescriptor(String defaultApplicationName, CommandLine cmd) { + + + YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor(); + if (!cmd.hasOption(CONTAINER.getOpt())) { // number of containers is required option! - LOG.error("Missing required argument " + CONTAINER.getOpt()); + LOG.error("Missing required argument {}", CONTAINER.getOpt()); printUsage(); - return null; + throw new IllegalArgumentException("Missing required argument " + CONTAINER.getOpt()); } - flinkYarnClient.setTaskManagerCount(Integer.valueOf(cmd.getOptionValue(CONTAINER.getOpt()))); + yarnClusterDescriptor.setTaskManagerCount(Integer.valueOf(cmd.getOptionValue(CONTAINER.getOpt()))); // Jar Path Path localJarPath; @@ -136,26 +216,13 @@ public AbstractFlinkYarnClient createFlinkYarnClient(CommandLine cmd) { } 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()); - } - - flinkYarnClient.setLocalJarPath(localJarPath); - - // Conf Path - String confDirPath = CliFrontend.getConfigurationDirectoryFromEnv(); - GlobalConfiguration.loadConfiguration(confDirPath); - Configuration flinkConfiguration = GlobalConfiguration.getConfiguration(); - flinkYarnClient.setFlinkConfiguration(flinkConfiguration); - 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; + LOG.info("No path for the flink jar passed. Using the location of " + + yarnClusterDescriptor.getClass() + " to locate the jar"); + localJarPath = new Path("file://" + + yarnClusterDescriptor.getClass().getProtectionDomain().getCodeSource().getLocation().getPath()); } - Path confPath = new Path(confFile.getAbsolutePath()); - flinkYarnClient.setConfigurationFilePath(confPath); + yarnClusterDescriptor.setLocalJarPath(localJarPath); List shipFiles = new ArrayList<>(); // path to directory to ship @@ -174,75 +241,87 @@ public boolean accept(File dir, String name) { } } - //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.setFlinkLoggingConfigurationPath(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.warn("The configuration directory ('" + confDirPath + "') contains both LOG4J and " + - "Logback configuration files. Please delete or rename one of them."); - } // else - flinkYarnClient.setFlinkLoggingConfigurationPath(new Path(log4j.toURI())); - } - } - - flinkYarnClient.setShipFiles(shipFiles); + yarnClusterDescriptor.setShipFiles(shipFiles); // queue if (cmd.hasOption(QUEUE.getOpt())) { - flinkYarnClient.setQueue(cmd.getOptionValue(QUEUE.getOpt())); + yarnClusterDescriptor.setQueue(cmd.getOptionValue(QUEUE.getOpt())); } // JobManager Memory if (cmd.hasOption(JM_MEMORY.getOpt())) { int jmMemory = Integer.valueOf(cmd.getOptionValue(JM_MEMORY.getOpt())); - flinkYarnClient.setJobManagerMemory(jmMemory); + yarnClusterDescriptor.setJobManagerMemory(jmMemory); } // Task Managers memory if (cmd.hasOption(TM_MEMORY.getOpt())) { int tmMemory = Integer.valueOf(cmd.getOptionValue(TM_MEMORY.getOpt())); - flinkYarnClient.setTaskManagerMemory(tmMemory); + yarnClusterDescriptor.setTaskManagerMemory(tmMemory); } if (cmd.hasOption(SLOTS.getOpt())) { int slots = Integer.valueOf(cmd.getOptionValue(SLOTS.getOpt())); - flinkYarnClient.setTaskManagerSlots(slots); + yarnClusterDescriptor.setTaskManagerSlots(slots); } String[] dynamicProperties = null; if (cmd.hasOption(DYNAMIC_PROPERTIES.getOpt())) { dynamicProperties = cmd.getOptionValues(DYNAMIC_PROPERTIES.getOpt()); } - String dynamicPropertiesEncoded = StringUtils.join(dynamicProperties, - CliFrontend.YARN_DYNAMIC_PROPERTIES_SEPARATOR); + String dynamicPropertiesEncoded = StringUtils.join(dynamicProperties, YARN_DYNAMIC_PROPERTIES_SEPARATOR); - flinkYarnClient.setDynamicPropertiesEncoded(dynamicPropertiesEncoded); + yarnClusterDescriptor.setDynamicPropertiesEncoded(dynamicPropertiesEncoded); - if (cmd.hasOption(DETACHED.getOpt())) { + if (cmd.hasOption(DETACHED.getOpt()) || cmd.hasOption(CliFrontendParser.DETACHED_OPTION.getOpt())) { this.detachedMode = true; - flinkYarnClient.setDetachedMode(detachedMode); + yarnClusterDescriptor.setDetachedMode(true); } if(cmd.hasOption(NAME.getOpt())) { - flinkYarnClient.setName(cmd.getOptionValue(NAME.getOpt())); + yarnClusterDescriptor.setName(cmd.getOptionValue(NAME.getOpt())); } else { // set the default application name, if none is specified if(defaultApplicationName != null) { - flinkYarnClient.setName(defaultApplicationName); + yarnClusterDescriptor.setName(defaultApplicationName); } } - return flinkYarnClient; + // ----- Convenience ----- + + // the number of slots available from YARN: + int yarnTmSlots = yarnClusterDescriptor.getTaskManagerSlots(); + if (yarnTmSlots == -1) { + yarnTmSlots = 1; + } + + int maxSlots = yarnTmSlots * yarnClusterDescriptor.getTaskManagerCount(); + int userParallelism = Integer.valueOf(cmd.getOptionValue(CliFrontendParser.PARALLELISM_OPTION.getOpt(), "-1")); + if (userParallelism != -1) { + int slotsPerTM = userParallelism / yarnClusterDescriptor.getTaskManagerCount(); + String message = "The YARN cluster has " + maxSlots + " slots available, " + + "but the user requested a parallelism of " + userParallelism + " on YARN. " + + "Each of the " + yarnClusterDescriptor.getTaskManagerCount() + " TaskManagers " + + "will get "+slotsPerTM+" slots."; + logAndSysout(message); + yarnClusterDescriptor.setTaskManagerSlots(slotsPerTM); + } + + return yarnClusterDescriptor; } + @Override + public YarnClusterClient createClient(String applicationName, CommandLine cmdLine) throws Exception { + + YarnClusterDescriptor yarnClusterDescriptor = createDescriptor(applicationName, cmdLine); + + try { + return yarnClusterDescriptor.deploy(); + } catch (Exception e) { + throw new RuntimeException("Error deploying the YARN cluster", e); + } + + } private void printUsage() { System.out.println("Usage:"); @@ -268,23 +347,6 @@ private void printUsage() { formatter.printHelp(" ", opt); } - public static AbstractFlinkYarnClient getFlinkYarnClient() { - AbstractFlinkYarnClient yarnClient; - try { - Class yarnClientClass = - Class.forName("org.apache.flink.yarn.FlinkYarnClient").asSubclass(AbstractFlinkYarnClient.class); - yarnClient = InstantiationUtil.instantiate(yarnClientClass, AbstractFlinkYarnClient.class); - } - catch (ClassNotFoundException e) { - System.err.println("Unable to locate the Flink YARN Client. " + - "Please ensure that you are using a Flink build with Hadoop2/YARN support. Message: " + - e.getMessage()); - e.printStackTrace(System.err); - return null; // make it obvious - } - return yarnClient; - } - private static void writeYarnProperties(Properties properties, File propertiesFile) { try { OutputStream out = new FileOutputStream(propertiesFile); @@ -296,7 +358,7 @@ private static void writeYarnProperties(Properties properties, File propertiesFi propertiesFile.setReadable(true, false); // readable for all. } - public static void runInteractiveCli(AbstractFlinkYarnCluster yarnCluster, boolean readConsoleInput) { + public static void runInteractiveCli(YarnClusterClient yarnCluster, boolean readConsoleInput) { final String HELP = "Available commands:\n" + "help - show these commands\n" + "stop - stop the YARN session"; @@ -325,9 +387,9 @@ public static void runInteractiveCli(AbstractFlinkYarnCluster yarnCluster, boole } } - if (yarnCluster.hasFailed()) { + if (yarnCluster.getApplicationStatus() != ApplicationStatus.SUCCEEDED) { System.err.println("The YARN cluster has failed"); - yarnCluster.shutdown(true); + yarnCluster.shutdown(); } // wait until CLIENT_POLLING_INTERVAL is over or the user entered something. @@ -355,7 +417,7 @@ public static void runInteractiveCli(AbstractFlinkYarnCluster yarnCluster, boole } } - if (yarnCluster.hasBeenStopped()) { + if (yarnCluster.hasBeenShutdown()) { LOG.info("Stopping interactive command line interface, YARN cluster has been stopped."); break; } @@ -366,11 +428,16 @@ public static void runInteractiveCli(AbstractFlinkYarnCluster yarnCluster, boole } public static void main(String[] args) { - FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", "", true); // no prefix for the YARN session + FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", ""); // no prefix for the YARN session System.exit(cli.run(args)); } - public void getYARNSessionCLIOptions(Options options) { + @Override + public String getIdentifier() { + return ID; + } + + public void addOptions(Options options) { options.addOption(FLINK_JAR); options.addOption(JM_MEMORY); options.addOption(TM_MEMORY); @@ -385,12 +452,22 @@ public void getYARNSessionCLIOptions(Options options) { options.addOption(NAME); } + @Override + public ClusterClient retrieveCluster(Configuration config) throws Exception { + + if(resumeFromYarnProperties(config)) { + return new StandaloneClusterClient(config); + } + + return null; + } + public int run(String[] args) { // // Command Line Options // Options options = new Options(); - getYARNSessionCLIOptions(options); + addOptions(options); CommandLineParser parser = new PosixParser(); CommandLine cmd; @@ -404,7 +481,7 @@ public int run(String[] args) { // Query cluster for metrics if (cmd.hasOption(QUERY.getOpt())) { - AbstractFlinkYarnClient flinkYarnClient = getFlinkYarnClient(); + YarnClusterDescriptor flinkYarnClient = new YarnClusterDescriptor(); String description; try { description = flinkYarnClient.getClusterDescription(); @@ -416,61 +493,60 @@ public int run(String[] args) { System.out.println(description); return 0; } else { - AbstractFlinkYarnClient flinkYarnClient = createFlinkYarnClient(cmd); - if (flinkYarnClient == null) { + YarnClusterDescriptor flinkYarnClient; + try { + flinkYarnClient = createDescriptor(null, cmd); + } catch (Exception e) { System.err.println("Error while starting the YARN Client. Please check log output!"); return 1; } try { yarnCluster = flinkYarnClient.deploy(); - // only connect to cluster if its not a detached session. - if(!flinkYarnClient.isDetached()) { - yarnCluster.connectToCluster(); - } } catch (Exception e) { System.err.println("Error while deploying YARN cluster: "+e.getMessage()); e.printStackTrace(System.err); return 1; } - //------------------ Cluster deployed, handle connection details + //------------------ ClusterClient deployed, handle connection details String jobManagerAddress = yarnCluster.getJobManagerAddress().getAddress().getHostAddress() + ":" + yarnCluster.getJobManagerAddress().getPort(); System.out.println("Flink JobManager is now running on " + jobManagerAddress); System.out.println("JobManager Web Interface: " + yarnCluster.getWebInterfaceURL()); // file that we write into the conf/ dir containing the jobManager address and the dop. - File yarnPropertiesFile = new File(CliFrontend.getYarnPropertiesLocation(yarnCluster.getFlinkConfiguration())); + File yarnPropertiesFile = new File(getYarnPropertiesLocation(yarnCluster.getFlinkConfiguration())); Properties yarnProps = new Properties(); - yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_JOBMANAGER_KEY, jobManagerAddress); + yarnProps.setProperty(YARN_PROPERTIES_JOBMANAGER_KEY, jobManagerAddress); if (flinkYarnClient.getTaskManagerSlots() != -1) { String parallelism = Integer.toString(flinkYarnClient.getTaskManagerSlots() * flinkYarnClient.getTaskManagerCount()); - yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_PARALLELISM, parallelism); + yarnProps.setProperty(YARN_PROPERTIES_PARALLELISM, parallelism); } // add dynamic properties if (flinkYarnClient.getDynamicPropertiesEncoded() != null) { - yarnProps.setProperty(CliFrontend.YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING, + yarnProps.setProperty(YARN_PROPERTIES_DYNAMIC_PROPERTIES_STRING, flinkYarnClient.getDynamicPropertiesEncoded()); } writeYarnProperties(yarnProps, yarnPropertiesFile); - //------------------ Cluster running, let user control it ------------ + //------------------ ClusterClient running, let user control it ------------ if (detachedMode) { // print info and quit: LOG.info("The Flink YARN client has been started in detached mode. In order to stop " + "Flink on YARN, use the following command or a YARN web interface to stop it:\n" + - "yarn application -kill "+yarnCluster.getApplicationId()+"\n" + + "yarn application -kill " + yarnCluster.getClusterIdentifier() + "\n" + "Please also note that the temporary files of the YARN session in {} will not be removed.", flinkYarnClient.getSessionFilesDir()); + yarnCluster.disconnect(); } else { runInteractiveCli(yarnCluster, acceptInteractiveInput); - if (!yarnCluster.hasBeenStopped()) { + if (!yarnCluster.hasBeenShutdown()) { LOG.info("Command Line Interface requested session shutdown"); - yarnCluster.shutdown(false); + yarnCluster.shutdown(); } try { @@ -483,23 +559,48 @@ public int run(String[] args) { return 0; } - /** - * Sets the default Yarn Application Name. - * @param defaultApplicationName the name of the yarn application to use - * @return FlinkYarnSessionCli instance, for chaining - */ - public FlinkYarnSessionCli withDefaultApplicationName(String defaultApplicationName) { - this.defaultApplicationName = defaultApplicationName; - return this; - } - /** * Utility method for tests. */ public void stop() { if (yarnCluster != null) { LOG.info("Command line interface is shutting down the yarnCluster"); - yarnCluster.shutdown(false); + yarnCluster.shutdown(); } } + + private void logAndSysout(String message) { + LOG.info(message); + System.out.println(message); + } + + public static Map getDynamicProperties(String dynamicPropertiesEncoded) { + if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) { + Map properties = new HashMap<>(); + + String[] propertyLines = dynamicPropertiesEncoded.split(YARN_DYNAMIC_PROPERTIES_SEPARATOR); + for (String propLine : propertyLines) { + if (propLine == null) { + continue; + } + + String[] kv = propLine.split("="); + if (kv.length >= 2 && kv[0] != null && kv[1] != null && kv[0].length() > 0) { + properties.put(kv[0], kv[1]); + } + } + return properties; + } + else { + return Collections.emptyMap(); + } + } + + private static String getYarnPropertiesLocation(Configuration conf) { + String defaultPropertiesFileLocation = System.getProperty("java.io.tmpdir"); + String currentUser = System.getProperty("user.name"); + String propertiesFileLocation = conf.getString(ConfigConstants.YARN_PROPERTIES_FILE_LOCATION, defaultPropertiesFileLocation); + + return propertiesFileLocation + File.separator + YARN_PROPERTIES_FILE + currentUser; + } } diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala index 2876309b3dd42..aea1aac836753 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationClient.scala @@ -23,12 +23,10 @@ import java.util.UUID import akka.actor._ import grizzled.slf4j.Logger import org.apache.flink.configuration.Configuration -import org.apache.flink.runtime.clusterframework.ApplicationStatus import org.apache.flink.runtime.clusterframework.messages._ import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService} import org.apache.flink.runtime.{LeaderSessionMessageFilter, FlinkActor, LogMessages} import org.apache.flink.yarn.YarnMessages._ -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus import scala.collection.mutable import scala.concurrent.duration._ @@ -36,7 +34,7 @@ import scala.language.postfixOps /** Actor which is responsible to repeatedly poll the Yarn cluster status from the ResourceManager. * - * This class represents the bridge between the [[FlinkYarnCluster]] and the + * This class represents the bridge between the [[YarnClusterClient]] and the * [[YarnApplicationMasterRunner]]. * * @param flinkConfig Configuration object @@ -135,9 +133,9 @@ class ApplicationClient( } case msg: RegisterInfoMessageListenerSuccessful => + // The job manager acts as a proxy between the client and the resource managert val jm = sender() - - log.info(s"Successfully registered at the ResourceManager $jm") + log.info(s"Successfully registered at the ResourceManager using JobManager $jm") yarnJobManager = Some(jm) diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala index 8645581dff467..da1917bc59a35 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnMessages.scala @@ -18,12 +18,13 @@ package org.apache.flink.yarn -import java.util.{List => JavaList, UUID, Date} +import java.util.{Date, UUID, List => JavaList} import org.apache.flink.api.common.JobID +import org.apache.flink.runtime.clusterframework.ApplicationStatus import org.apache.flink.runtime.messages.RequiresLeaderSessionID import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.records.{ContainerStatus, Container, FinalApplicationStatus} +import org.apache.hadoop.yarn.api.records.{Container, ContainerStatus, FinalApplicationStatus} import scala.concurrent.duration.{Deadline, FiniteDuration} @@ -31,7 +32,7 @@ object YarnMessages { case class ApplicationMasterStatus(numTaskManagers: Int, numSlots: Int) - case class LocalStopYarnSession(status: FinalApplicationStatus, diagnostics: String) + case class LocalStopYarnSession(status: ApplicationStatus, diagnostics: String) /** * Entry point to start a new YarnSession.