From 2910e863f0557dcc30f9934251b1e8c5de9fb001 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 7 Dec 2017 13:57:24 +0100 Subject: [PATCH 01/12] [FLINK-8328] [flip6] Move Yarn ApplicationStatus polling out of YarnClusterClient Introduce YarnApplicationStatusMonitor which does the Yarn ApplicationStatus polling in the FlinkYarnSessionCli. This decouples the YarnClusterClient from the actual communication with Yarn and, thus, gives a better separation of concerns. --- ...iFrontendYarnAddressConfigurationTest.java | 2 +- .../yarn/AbstractYarnClusterDescriptor.java | 2 +- .../apache/flink/yarn/YarnClusterClient.java | 110 +------ .../flink/yarn/cli/FlinkYarnSessionCli.java | 285 +++++++++++------- .../cli/YarnApplicationStatusMonitor.java | 101 +++++++ 5 files changed, 288 insertions(+), 212 deletions(-) create mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java 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 index 1fed554c025ea..1b457a572490a 100644 --- 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 @@ -379,7 +379,7 @@ public TestingYarnClusterDescriptor(Configuration flinkConfiguration, String con } @Override - protected YarnClient getYarnClient() { + public YarnClient getYarnClient() { return new TestYarnClient(); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index a9101488037b9..86ddd9b4f4b1b 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -308,7 +308,7 @@ public void setZookeeperNamespace(String zookeeperNamespace) { * Gets a Hadoop Yarn client. * @return Returns a YarnClient which has to be shutdown manually */ - protected YarnClient getYarnClient() { + public YarnClient getYarnClient() { YarnClient yarnClient = YarnClient.createYarnClient(); yarnClient.init(conf); yarnClient.start(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java index ceca29d5d7f25..80d09436ed6b1 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java @@ -40,7 +40,6 @@ import akka.actor.Props; import akka.pattern.Patterns; import akka.util.Timeout; -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.YarnApplicationState; @@ -68,12 +67,9 @@ public class YarnClusterClient extends ClusterClient { private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClient.class); - private static final int POLLING_THREAD_INTERVAL_MS = 1000; - private YarnClient yarnClient; private Thread clientShutdownHook = new ClientShutdownHook(); - private PollingThread pollingRunner; //---------- Class internal fields ------------------- @@ -130,10 +126,6 @@ public YarnClusterClient( actorSystemLoader, highAvailabilityServices); - this.pollingRunner = new PollingThread(yarnClient, appId); - this.pollingRunner.setDaemon(true); - this.pollingRunner.start(); - Runtime.getRuntime().addShutdownHook(clientShutdownHook); } @@ -158,14 +150,6 @@ public void disconnect() { // we are already in the shutdown hook } - try { - pollingRunner.stopRunner(); - pollingRunner.join(1000); - } catch (InterruptedException e) { - LOG.warn("Shutdown of the polling runner was interrupted", e); - Thread.currentThread().interrupt(); - } - isConnected = false; } @@ -254,34 +238,6 @@ public GetClusterStatusResponse getClusterStatus() { } } - public ApplicationStatus getApplicationStatus() { - if (!isConnected) { - throw new IllegalStateException("The cluster has been connected to the ApplicationMaster."); - } - ApplicationReport lastReport = null; - if (pollingRunner == null) { - LOG.warn("YarnClusterClient.getApplicationStatus() has been called on an uninitialized cluster." + - "The system might be in an erroneous state"); - } else { - lastReport = pollingRunner.getLastReport(); - } - if (lastReport == null) { - 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 ApplicationStatus.UNKNOWN; - } else { - YarnApplicationState appState = lastReport.getYarnApplicationState(); - 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()); - } - return status; - } - } - @Override public List getNewMessages() { @@ -371,7 +327,7 @@ public void shutdownCluster() { try { Future response = Patterns.ask(applicationClient.get(), - new YarnMessages.LocalStopYarnSession(getApplicationStatus(), + new YarnMessages.LocalStopYarnSession(ApplicationStatus.CANCELED, "Flink YARN Client requested shutdown"), new Timeout(akkaDuration)); Await.ready(response, akkaDuration); @@ -392,14 +348,6 @@ public void shutdownCluster() { LOG.warn("Exception while deleting the JobManager address file", e); } - try { - pollingRunner.stopRunner(); - pollingRunner.join(1000); - } catch (InterruptedException e) { - LOG.warn("Shutdown of the polling runner was interrupted", e); - Thread.currentThread().interrupt(); - } - try { ApplicationReport appReport = yarnClient.getApplicationReport(appId); @@ -443,62 +391,6 @@ public void run() { } } - // -------------------------- Polling ------------------------ - - private static class PollingThread extends Thread { - - AtomicBoolean running = new AtomicBoolean(true); - private YarnClient yarnClient; - private ApplicationId appId; - - // ------- status information stored in the polling thread - private final Object lock = new Object(); - private ApplicationReport lastReport; - - public PollingThread(YarnClient yarnClient, ApplicationId appId) { - this.yarnClient = yarnClient; - this.appId = appId; - } - - public void stopRunner() { - if (!running.get()) { - LOG.warn("Polling thread was already stopped"); - } - running.set(false); - } - - public ApplicationReport getLastReport() { - synchronized (lock) { - return lastReport; - } - } - - @Override - public void run() { - while (running.get() && yarnClient.isInState(Service.STATE.STARTED)) { - try { - ApplicationReport report = yarnClient.getApplicationReport(appId); - synchronized (lock) { - lastReport = report; - } - } catch (Exception e) { - LOG.warn("Error while getting application report", e); - } - try { - 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)) { - // == if the polling thread is still running but the yarn client is stopped. - LOG.warn("YARN client is unexpected in state " + yarnClient.getServiceState()); - } - } - } - @Override public boolean isDetached() { return super.isDetached() || clusterDescriptor.isDetachedMode(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index df4ef1fc4ae71..13fe5a116895a 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -31,8 +31,10 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; +import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterClient; @@ -71,6 +73,8 @@ import java.util.Map; import java.util.Properties; import java.util.concurrent.Callable; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import static org.apache.flink.client.cli.CliFrontendParser.ADDRESS_OPTION; import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_ID; @@ -86,7 +90,7 @@ public class FlinkYarnSessionCli implements CustomCommandLine 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; + private static final long CLIENT_POLLING_INTERVAL_MS = 3000L; /** The id for the CommandLine interface. */ private static final String ID = "yarn-cluster"; @@ -99,6 +103,10 @@ public class FlinkYarnSessionCli implements CustomCommandLine private static final String YARN_DYNAMIC_PROPERTIES_SEPARATOR = "@@"; // this has to be a regex for String.split() + private static final String YARN_SESSION_HELP = "Available commands:\n" + + "help - show these commands\n" + + "stop - stop the YARN session"; + //------------------------------------ Command Line argument options ------------------------- // the prefix transformation is used by the CliFrontend static constructor. private final Option query; @@ -419,104 +427,6 @@ private void printUsage() { formatter.printHelp(" ", options); } - private static void writeYarnProperties(Properties properties, File propertiesFile) { - try (final OutputStream out = new FileOutputStream(propertiesFile)) { - properties.store(out, "Generated YARN properties file"); - } catch (IOException e) { - throw new RuntimeException("Error writing the properties file", e); - } - propertiesFile.setReadable(true, false); // readable for all. - } - - public static void runInteractiveCli(YarnClusterClient yarnCluster, boolean readConsoleInput) { - final String help = "Available commands:\n" + - "help - show these commands\n" + - "stop - stop the YARN session"; - int numTaskmanagers = 0; - try { - BufferedReader in = new BufferedReader(new InputStreamReader(System.in)); - label: - while (true) { - // ------------------ check if there are updates by the cluster ----------- - - try { - GetClusterStatusResponse status = yarnCluster.getClusterStatus(); - LOG.debug("Received status message: {}", status); - - if (status != null && numTaskmanagers != status.numRegisteredTaskManagers()) { - System.err.println("Number of connected TaskManagers changed to " + - status.numRegisteredTaskManagers() + ". " + - "Slots available: " + status.totalNumberOfSlots()); - numTaskmanagers = status.numRegisteredTaskManagers(); - } - } catch (Exception e) { - LOG.warn("Could not retrieve the current cluster status. Skipping current retrieval attempt ...", e); - } - - List messages = yarnCluster.getNewMessages(); - if (messages != null && messages.size() > 0) { - System.err.println("New messages from the YARN cluster: "); - for (String msg : messages) { - System.err.println(msg); - } - } - - if (yarnCluster.getApplicationStatus() != ApplicationStatus.SUCCEEDED) { - System.err.println("The YARN cluster has failed"); - yarnCluster.shutdown(); - } - - // wait until CLIENT_POLLING_INTERVAL is over or the user entered something. - long startTime = System.currentTimeMillis(); - while ((System.currentTimeMillis() - startTime) < CLIENT_POLLING_INTERVALL * 1000 - && (!readConsoleInput || !in.ready())) { - Thread.sleep(200); - } - //------------- handle interactive command by user. ---------------------- - - if (readConsoleInput && in.ready()) { - String command = in.readLine(); - switch (command) { - case "quit": - case "stop": - yarnCluster.shutdownCluster(); - break label; - - case "help": - System.err.println(help); - break; - default: - System.err.println("Unknown command '" + command + "'. Showing help: \n" + help); - break; - } - } - - if (yarnCluster.hasBeenShutdown()) { - LOG.info("Stopping interactive command line interface, YARN cluster has been stopped."); - break; - } - } - } catch (Exception e) { - LOG.warn("Exception while running the interactive command line interface", e); - } - } - - public static void main(final String[] args) throws Exception { - final FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", ""); // no prefix for the YARN session - - final String configurationDirectory = CliFrontend.getConfigurationDirectoryFromEnv(); - - final Configuration flinkConfiguration = GlobalConfiguration.loadConfiguration(); - SecurityUtils.install(new SecurityConfiguration(flinkConfiguration)); - int retCode = SecurityUtils.getInstalledContext().runSecured(new Callable() { - @Override - public Integer call() { - return cli.run(args, flinkConfiguration, configurationDirectory); - } - }); - System.exit(retCode); - } - @Override public boolean isActive(CommandLine commandLine, Configuration configuration) { String jobManagerOption = commandLine.getOptionValue(ADDRESS_OPTION.getOpt(), null); @@ -660,7 +570,25 @@ public int run( "yarn application -kill " + applicationId.getOpt()); yarnCluster.disconnect(); } else { - runInteractiveCli(yarnCluster, true); + ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1); + + try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor( + yarnDescriptor.getYarnClient(), + yarnCluster.getApplicationId(), + new ScheduledExecutorServiceAdapter(scheduledExecutorService))) { + runInteractiveCli( + yarnCluster, + yarnApplicationStatusMonitor, + true); + } catch (Exception e) { + LOG.info("Could not properly close the Yarn application status monitor.", e); + } finally { + // shut down the scheduled executor service + ExecutorUtils.gracefulShutdown( + 1000L, + TimeUnit.MILLISECONDS, + scheduledExecutorService); + } } } else { @@ -717,7 +645,26 @@ public int run( yarnCluster.waitForClusterToBeReady(); yarnCluster.disconnect(); } else { - runInteractiveCli(yarnCluster, acceptInteractiveInput); + + ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1); + + try (YarnApplicationStatusMonitor yarnApplicationStatusMonitor = new YarnApplicationStatusMonitor( + yarnDescriptor.getYarnClient(), + yarnCluster.getApplicationId(), + new ScheduledExecutorServiceAdapter(scheduledExecutorService))){ + runInteractiveCli( + yarnCluster, + yarnApplicationStatusMonitor, + acceptInteractiveInput); + } catch (Exception e) { + LOG.info("Could not properly close the Yarn application status monitor.", e); + } finally { + // shut down the scheduled executor service + ExecutorUtils.gracefulShutdown( + 1000L, + TimeUnit.MILLISECONDS, + scheduledExecutorService); + } } } return 0; @@ -743,6 +690,142 @@ private void logAndSysout(String message) { System.out.println(message); } + public static void main(final String[] args) throws Exception { + final FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", ""); // no prefix for the YARN session + + final String configurationDirectory = CliFrontend.getConfigurationDirectoryFromEnv(); + + final Configuration flinkConfiguration = GlobalConfiguration.loadConfiguration(); + SecurityUtils.install(new SecurityConfiguration(flinkConfiguration)); + int retCode = SecurityUtils.getInstalledContext().runSecured(new Callable() { + @Override + public Integer call() { + return cli.run(args, flinkConfiguration, configurationDirectory); + } + }); + System.exit(retCode); + } + + private static void runInteractiveCli( + YarnClusterClient clusterClient, + YarnApplicationStatusMonitor yarnApplicationStatusMonitor, + boolean readConsoleInput) { + try (BufferedReader in = new BufferedReader(new InputStreamReader(System.in))) { + boolean continueRepl = true; + int numTaskmanagers = 0; + long unknownStatusSince = System.currentTimeMillis(); + + while (continueRepl) { + + final ApplicationStatus applicationStatus = yarnApplicationStatusMonitor.getApplicationStatusNow(); + + switch (applicationStatus) { + case FAILED: + case CANCELED: + System.err.println("The Flink Yarn cluster has failed."); + continueRepl = false; + break; + case UNKNOWN: + if (unknownStatusSince < 0L) { + unknownStatusSince = System.currentTimeMillis(); + } + + if ((System.currentTimeMillis() - unknownStatusSince) > CLIENT_POLLING_INTERVAL_MS) { + System.err.println("The Flink Yarn cluster is in an unknown state. Please check the Yarn cluster."); + continueRepl = false; + } else { + continueRepl = repStep(in, readConsoleInput); + } + break; + case SUCCEEDED: + if (unknownStatusSince > 0L) { + unknownStatusSince = -1L; + } + + // ------------------ check if there are updates by the cluster ----------- + try { + final GetClusterStatusResponse status = clusterClient.getClusterStatus(); + + if (status != null && numTaskmanagers != status.numRegisteredTaskManagers()) { + System.err.println("Number of connected TaskManagers changed to " + + status.numRegisteredTaskManagers() + ". " + + "Slots available: " + status.totalNumberOfSlots()); + numTaskmanagers = status.numRegisteredTaskManagers(); + } + } catch (Exception e) { + LOG.warn("Could not retrieve the current cluster status. Skipping current retrieval attempt ...", e); + } + + printClusterMessages(clusterClient); + + continueRepl = repStep(in, readConsoleInput); + } + } + } catch (Exception e) { + LOG.warn("Exception while running the interactive command line interface.", e); + } + } + + private static void printClusterMessages(YarnClusterClient clusterClient) { + final List messages = clusterClient.getNewMessages(); + if (messages != null && messages.size() > 0) { + System.err.println("New messages from the YARN cluster: "); + for (String msg : messages) { + System.err.println(msg); + } + } + } + + /** + * Read-Evaluate-Print step for the REPL. + * + * @param in to read from + * @param readConsoleInput true if console input has to be read + * @return true if the REPL shall be continued, otherwise false + * @throws IOException + * @throws InterruptedException + */ + private static boolean repStep( + BufferedReader in, + boolean readConsoleInput) throws IOException, InterruptedException { + + // wait until CLIENT_POLLING_INTERVAL is over or the user entered something. + long startTime = System.currentTimeMillis(); + while ((System.currentTimeMillis() - startTime) < CLIENT_POLLING_INTERVAL_MS + && (!readConsoleInput || !in.ready())) { + Thread.sleep(200L); + } + //------------- handle interactive command by user. ---------------------- + + if (readConsoleInput && in.ready()) { + String command = in.readLine(); + switch (command) { + case "quit": + case "stop": + return false; + + case "help": + System.err.println(YARN_SESSION_HELP); + break; + default: + System.err.println("Unknown command '" + command + "'. Showing help:"); + System.err.println(YARN_SESSION_HELP); + break; + } + } + + return true; + } + + private static void writeYarnProperties(Properties properties, File propertiesFile) { + try (final OutputStream out = new FileOutputStream(propertiesFile)) { + properties.store(out, "Generated YARN properties file"); + } catch (IOException e) { + throw new RuntimeException("Error writing the properties file", e); + } + propertiesFile.setReadable(true, false); // readable for all. + } + public static Map getDynamicProperties(String dynamicPropertiesEncoded) { if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) { Map properties = new HashMap<>(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java new file mode 100644 index 0000000000000..81659ba3dd85a --- /dev/null +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java @@ -0,0 +1,101 @@ +/* + * 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.cli; + +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.util.Preconditions; + +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.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + * Utility class which monitors the specified yarn application status periodically. + */ +public class YarnApplicationStatusMonitor implements AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(YarnApplicationStatusMonitor.class); + + private static final long UPDATE_INTERVAL = 1000L; + + private final YarnClient yarnClient; + + private final ApplicationId yarnApplicationId; + + private final ScheduledFuture applicationStatusUpdateFuture; + + private volatile ApplicationStatus applicationStatus; + + public YarnApplicationStatusMonitor( + YarnClient yarnClient, + ApplicationId yarnApplicationId, + ScheduledExecutor scheduledExecutor) { + this.yarnClient = Preconditions.checkNotNull(yarnClient); + this.yarnApplicationId = Preconditions.checkNotNull(yarnApplicationId); + + applicationStatusUpdateFuture = scheduledExecutor.scheduleWithFixedDelay( + this::updateApplicationStatus, + UPDATE_INTERVAL, + UPDATE_INTERVAL, + TimeUnit.MILLISECONDS); + + applicationStatus = ApplicationStatus.UNKNOWN; + } + + public ApplicationStatus getApplicationStatusNow() { + return applicationStatus; + } + + @Override + public void close() throws Exception { + applicationStatusUpdateFuture.cancel(false); + } + + private void updateApplicationStatus() { + if (yarnClient.isInState(Service.STATE.STARTED)) { + final ApplicationReport applicationReport; + + try { + applicationReport = yarnClient.getApplicationReport(yarnApplicationId); + } catch (Exception e) { + LOG.info("Could not retrieve the Yarn application report for {}.", yarnApplicationId); + return; + } + + YarnApplicationState yarnApplicationState = applicationReport.getYarnApplicationState(); + + if (yarnApplicationState == YarnApplicationState.FAILED || yarnApplicationState == YarnApplicationState.KILLED) { + applicationStatus = ApplicationStatus.FAILED; + } else { + applicationStatus = ApplicationStatus.SUCCEEDED; + } + } else { + LOG.info("Yarn client is no longer in state STARTED. Stopping the Yarn application status monitor."); + applicationStatusUpdateFuture.cancel(false); + } + } +} From 368a94f614753658d7a6413ec2b6f0bbe97fdc68 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 10 Jan 2018 14:54:39 +0100 Subject: [PATCH 02/12] fixup! [FLINK-8328] [flip6] Move Yarn ApplicationStatus polling out of YarnClusterClient --- .../flink/yarn/cli/FlinkYarnSessionCli.java | 20 +++++++++---------- .../cli/YarnApplicationStatusMonitor.java | 2 +- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 13fe5a116895a..c31d923e839bf 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -580,8 +580,6 @@ public int run( yarnCluster, yarnApplicationStatusMonitor, true); - } catch (Exception e) { - LOG.info("Could not properly close the Yarn application status monitor.", e); } finally { // shut down the scheduled executor service ExecutorUtils.gracefulShutdown( @@ -656,8 +654,6 @@ public int run( yarnCluster, yarnApplicationStatusMonitor, acceptInteractiveInput); - } catch (Exception e) { - LOG.info("Could not properly close the Yarn application status monitor.", e); } finally { // shut down the scheduled executor service ExecutorUtils.gracefulShutdown( @@ -713,7 +709,8 @@ private static void runInteractiveCli( try (BufferedReader in = new BufferedReader(new InputStreamReader(System.in))) { boolean continueRepl = true; int numTaskmanagers = 0; - long unknownStatusSince = System.currentTimeMillis(); + boolean isLastStatusUnknown = true; + long unknownStatusSince = System.nanoTime(); while (continueRepl) { @@ -726,11 +723,12 @@ private static void runInteractiveCli( continueRepl = false; break; case UNKNOWN: - if (unknownStatusSince < 0L) { - unknownStatusSince = System.currentTimeMillis(); + if (!isLastStatusUnknown) { + unknownStatusSince = System.nanoTime(); + isLastStatusUnknown = true; } - if ((System.currentTimeMillis() - unknownStatusSince) > CLIENT_POLLING_INTERVAL_MS) { + if ((System.nanoTime() - unknownStatusSince) > CLIENT_POLLING_INTERVAL_MS) { System.err.println("The Flink Yarn cluster is in an unknown state. Please check the Yarn cluster."); continueRepl = false; } else { @@ -738,8 +736,8 @@ private static void runInteractiveCli( } break; case SUCCEEDED: - if (unknownStatusSince > 0L) { - unknownStatusSince = -1L; + if (isLastStatusUnknown) { + isLastStatusUnknown = false; } // ------------------ check if there are updates by the cluster ----------- @@ -768,7 +766,7 @@ private static void runInteractiveCli( private static void printClusterMessages(YarnClusterClient clusterClient) { final List messages = clusterClient.getNewMessages(); - if (messages != null && messages.size() > 0) { + if (!messages.isEmpty()) { System.err.println("New messages from the YARN cluster: "); for (String msg : messages) { System.err.println(msg); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java index 81659ba3dd85a..88d77475afbfc 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnApplicationStatusMonitor.java @@ -71,7 +71,7 @@ public ApplicationStatus getApplicationStatusNow() { } @Override - public void close() throws Exception { + public void close() { applicationStatusUpdateFuture.cancel(false); } From 53d01033d55de563b0c84a4446f44a4273e0c1f1 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 11 Jan 2018 13:12:40 +0100 Subject: [PATCH 03/12] fixup! [FLINK-8328] [flip6] Move Yarn ApplicationStatus polling out of YarnClusterClient --- .../src/test/java/org/apache/flink/yarn/YarnTestBase.java | 2 +- .../java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 ae39d0add22d5..e0299aae1fb92 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 @@ -656,7 +656,7 @@ protected void runWithArgs(String[] args, String terminateAfterString, String[] throw new RuntimeException("Runner failed", runner.getRunnerError()); } Assert.assertTrue("During the timeout period of " + startTimeoutSeconds + " seconds the " + - "expected string did not show up", expectedStringSeen); + "expected string \"" + terminateAfterString + "\" did not show up.", expectedStringSeen); LOG.info("Test was successful"); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index c31d923e839bf..54837585eaeb8 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -728,7 +728,7 @@ private static void runInteractiveCli( isLastStatusUnknown = true; } - if ((System.nanoTime() - unknownStatusSince) > CLIENT_POLLING_INTERVAL_MS) { + if ((System.nanoTime() - unknownStatusSince) > 5L * CLIENT_POLLING_INTERVAL_MS * 1_000_000L) { System.err.println("The Flink Yarn cluster is in an unknown state. Please check the Yarn cluster."); continueRepl = false; } else { From 5215fac6ff6684ac2f0920533a8dfc1dafa9718a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 20 Dec 2017 16:43:21 +0100 Subject: [PATCH 04/12] [FLINK-8329] [flip6] Move YarnClient to AbstractYarnClusterDescriptor Moves the YarnClient from the YarnClusterClient to the AbstractYarnClusterDescriptor. This makes the latter responsible for the lifecycle management of the client and gives a better separation of concerns. --- .../client/deployment/ClusterDescriptor.java | 2 +- .../Flip6StandaloneClusterDescriptor.java | 5 + .../StandaloneClusterDescriptor.java | 5 + ...iFrontendYarnAddressConfigurationTest.java | 80 ++-- .../flink/yarn/FlinkYarnSessionCliTest.java | 6 +- .../yarn/TestingYarnClusterDescriptor.java | 7 +- .../org/apache/flink/yarn/YARNITCase.java | 12 +- .../flink/yarn/YARNSessionFIFOITCase.java | 16 +- .../flink/yarn/YarnClusterDescriptorTest.java | 94 ++-- .../yarn/AbstractYarnClusterDescriptor.java | 87 ++-- .../apache/flink/yarn/YarnClusterClient.java | 14 +- .../flink/yarn/YarnClusterDescriptor.java | 9 +- .../flink/yarn/YarnClusterDescriptorV2.java | 9 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 6 +- .../flink/yarn/YarnClusterDescriptorTest.java | 451 +++++++++--------- 15 files changed, 432 insertions(+), 371 deletions(-) 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 index a62ceffc997c6..16039308e2176 100644 --- 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 @@ -24,7 +24,7 @@ /** * A descriptor to deploy a cluster (e.g. Yarn or Mesos) and return a Client for Cluster communication. */ -public interface ClusterDescriptor { +public interface ClusterDescriptor extends AutoCloseable { /** * Returns a String containing details about the cluster (NodeManagers, available memory, ...). diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java index 9d88f597a4565..b8eb5344c7451 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java @@ -60,4 +60,9 @@ public RestClusterClient deploySessionCluster(ClusterSpecification clusterSpecif public RestClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) { throw new UnsupportedOperationException("Can't deploy a standalone FLIP-6 per-job cluster."); } + + @Override + public void close() throws Exception { + // nothing to do + } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java index 51e267ab36cb5..3808efa33457f 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java @@ -59,4 +59,9 @@ public StandaloneClusterClient deploySessionCluster(ClusterSpecification cluster public StandaloneClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) { throw new UnsupportedOperationException("Can't deploy a standalone per-job cluster."); } + + @Override + public void close() throws Exception { + // nothing to do + } } 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 index 1b457a572490a..56087a18404a5 100644 --- 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 @@ -38,7 +38,6 @@ 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.client.api.YarnClient; import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; import org.apache.hadoop.yarn.exceptions.YarnException; import org.junit.AfterClass; @@ -375,12 +374,10 @@ protected AbstractYarnClusterDescriptor getClusterDescriptor( private class TestingYarnClusterDescriptor extends YarnClusterDescriptor { public TestingYarnClusterDescriptor(Configuration flinkConfiguration, String configurationDirectory) { - super(flinkConfiguration, configurationDirectory); - } - - @Override - public YarnClient getYarnClient() { - return new TestYarnClient(); + super( + flinkConfiguration, + configurationDirectory, + new TestYarnClient(finalApplicationStatus)); } @Override @@ -388,52 +385,51 @@ protected YarnClusterClient createYarnClusterClient( AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, - YarnClient yarnClient, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws IOException, YarnException { return Mockito.mock(YarnClusterClient.class); } + } + } - private class TestYarnClient extends YarnClientImpl { - - private final List reports = new LinkedList<>(); - - TestYarnClient() { - { // a report that of our Yarn application we want to resume from - ApplicationReport report = Mockito.mock(ApplicationReport.class); - Mockito.when(report.getHost()).thenReturn(TEST_YARN_JOB_MANAGER_ADDRESS); - Mockito.when(report.getRpcPort()).thenReturn(TEST_YARN_JOB_MANAGER_PORT); - Mockito.when(report.getApplicationId()).thenReturn(TEST_YARN_APPLICATION_ID); - Mockito.when(report.getFinalApplicationStatus()).thenReturn(finalApplicationStatus); - this.reports.add(report); - } - { // a second report, just for noise - ApplicationReport report = Mockito.mock(ApplicationReport.class); - Mockito.when(report.getHost()).thenReturn("1.2.3.4"); - Mockito.when(report.getRpcPort()).thenReturn(-123); - Mockito.when(report.getApplicationId()).thenReturn(ApplicationId.newInstance(0, 0)); - Mockito.when(report.getFinalApplicationStatus()).thenReturn(finalApplicationStatus); - this.reports.add(report); - } - } + private static class TestYarnClient extends YarnClientImpl { - @Override - public List getApplications() throws YarnException, IOException { - return reports; - } + private final List reports = new LinkedList<>(); - @Override - public ApplicationReport getApplicationReport(ApplicationId appId) throws YarnException, IOException { - for (ApplicationReport report : reports) { - if (report.getApplicationId().equals(appId)) { - return report; - } - } - throw new YarnException(); + TestYarnClient(FinalApplicationStatus finalApplicationStatus) { + { // a report that of our Yarn application we want to resume from + ApplicationReport report = Mockito.mock(ApplicationReport.class); + Mockito.when(report.getHost()).thenReturn(TEST_YARN_JOB_MANAGER_ADDRESS); + Mockito.when(report.getRpcPort()).thenReturn(TEST_YARN_JOB_MANAGER_PORT); + Mockito.when(report.getApplicationId()).thenReturn(TEST_YARN_APPLICATION_ID); + Mockito.when(report.getFinalApplicationStatus()).thenReturn(finalApplicationStatus); + this.reports.add(report); + } + { // a second report, just for noise + ApplicationReport report = Mockito.mock(ApplicationReport.class); + Mockito.when(report.getHost()).thenReturn("1.2.3.4"); + Mockito.when(report.getRpcPort()).thenReturn(-123); + Mockito.when(report.getApplicationId()).thenReturn(ApplicationId.newInstance(0, 0)); + Mockito.when(report.getFinalApplicationStatus()).thenReturn(finalApplicationStatus); + this.reports.add(report); + } + } + + @Override + public List getApplications() throws YarnException, IOException { + return reports; + } + + @Override + public ApplicationReport getApplicationReport(ApplicationId appId) throws YarnException, IOException { + for (ApplicationReport report : reports) { + if (report.getApplicationId().equals(appId)) { + return report; } } + throw new YarnException(); } } } 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 8eef8f0696669..3fe8d2f75594a 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 @@ -174,7 +174,10 @@ public TestCLI(String shortPrefix, String longPrefix) { private static class JarAgnosticClusterDescriptor extends YarnClusterDescriptor { public JarAgnosticClusterDescriptor(Configuration flinkConfiguration, String configurationDirectory) { - super(flinkConfiguration, configurationDirectory); + super( + flinkConfiguration, + configurationDirectory, + YarnClient.createYarnClient()); } @Override @@ -202,7 +205,6 @@ public TestingYarnClusterClient( super(descriptor, numberTaskManagers, slotsPerTaskManager, - Mockito.mock(YarnClient.class), Mockito.mock(ApplicationReport.class), config, false); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java index 30d2798523566..e66d2e07ac0a4 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java @@ -23,6 +23,8 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.Preconditions; +import org.apache.hadoop.yarn.client.api.YarnClient; + import java.io.File; import java.io.FilenameFilter; import java.util.ArrayList; @@ -36,7 +38,10 @@ public class TestingYarnClusterDescriptor extends AbstractYarnClusterDescriptor { public TestingYarnClusterDescriptor(Configuration configuration, String configurationDirectory) { - super(configuration, configurationDirectory); + super( + configuration, + configurationDirectory, + YarnClient.createYarnClient()); 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/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java index bc28c5bd88ed2..d9b468dee31ca 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java @@ -28,6 +28,7 @@ import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; @@ -49,10 +50,14 @@ public static void setup() { @Ignore("The cluster cannot be stopped yet.") @Test - public void testPerJobMode() { + public void testPerJobMode() throws Exception { Configuration configuration = new Configuration(); configuration.setString(AkkaOptions.ASK_TIMEOUT, "30 s"); - YarnClusterDescriptorV2 yarnClusterDescriptorV2 = new YarnClusterDescriptorV2(configuration, System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR)); + final YarnClient yarnClient = YarnClient.createYarnClient(); + YarnClusterDescriptorV2 yarnClusterDescriptorV2 = new YarnClusterDescriptorV2( + configuration, + System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR), + yarnClient); yarnClusterDescriptorV2.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); yarnClusterDescriptorV2.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); @@ -78,6 +83,9 @@ public void testPerJobMode() { jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI())); YarnClusterClient clusterClient = yarnClusterDescriptorV2.deployJobCluster(clusterSpecification, jobGraph); + + clusterClient.shutdown(); + yarnClusterDescriptorV2.close(); } private static class InfiniteSource implements ParallelSourceFunction { 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 dd56f2f824372..fe44a15cdfcac 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 @@ -227,10 +227,14 @@ public void testJavaAPI() throws Exception { String confDirPath = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); Configuration configuration = GlobalConfiguration.loadConfiguration(); - AbstractYarnClusterDescriptor flinkYarnClient = new YarnClusterDescriptor(configuration, confDirPath); - Assert.assertNotNull("unable to get yarn client", flinkYarnClient); - flinkYarnClient.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); - flinkYarnClient.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); + final YarnClient yarnClient = YarnClient.createYarnClient(); + AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( + configuration, + confDirPath, + yarnClient); + Assert.assertNotNull("unable to get yarn client", clusterDescriptor); + clusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); + clusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() .setMasterMemoryMB(768) @@ -238,11 +242,10 @@ public void testJavaAPI() throws Exception { .setNumberTaskManagers(1) .setSlotsPerTaskManager(1) .createClusterSpecification(); - // deploy ClusterClient yarnCluster = null; try { - yarnCluster = flinkYarnClient.deploySessionCluster(clusterSpecification); + yarnCluster = clusterDescriptor.deploySessionCluster(clusterSpecification); } catch (Exception e) { LOG.warn("Failing test", e); Assert.fail("Error while deploying YARN cluster: " + e.getMessage()); @@ -272,6 +275,7 @@ public void testJavaAPI() throws Exception { LOG.info("Shutting down cluster. All tests passed"); // shutdown cluster yarnCluster.shutdown(); + clusterDescriptor.close(); LOG.info("Finished testJavaAPI()"); } } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java index f3e48c5050294..51445505c9b74 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java @@ -24,6 +24,7 @@ import org.apache.flink.util.TestLogger; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -50,32 +51,40 @@ public class YarnClusterDescriptorTest extends TestLogger { */ @Test public void testExplicitLibShipping() throws Exception { - AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(new Configuration(), temporaryFolder.getRoot().getAbsolutePath()); - descriptor.setLocalJarPath(new Path("/path/to/flink.jar")); + AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor( + new Configuration(), + temporaryFolder.getRoot().getAbsolutePath(), + YarnClient.createYarnClient()); - File libFile = temporaryFolder.newFile("libFile.jar"); - File libFolder = temporaryFolder.newFolder().getAbsoluteFile(); + try { + descriptor.setLocalJarPath(new Path("/path/to/flink.jar")); + + File libFile = temporaryFolder.newFile("libFile.jar"); + File libFolder = temporaryFolder.newFolder().getAbsoluteFile(); - Assert.assertFalse(descriptor.shipFiles.contains(libFile)); - Assert.assertFalse(descriptor.shipFiles.contains(libFolder)); + Assert.assertFalse(descriptor.shipFiles.contains(libFile)); + Assert.assertFalse(descriptor.shipFiles.contains(libFolder)); - List shipFiles = new ArrayList<>(); - shipFiles.add(libFile); - shipFiles.add(libFolder); + List shipFiles = new ArrayList<>(); + shipFiles.add(libFile); + shipFiles.add(libFolder); - descriptor.addShipFiles(shipFiles); + descriptor.addShipFiles(shipFiles); - Assert.assertTrue(descriptor.shipFiles.contains(libFile)); - Assert.assertTrue(descriptor.shipFiles.contains(libFolder)); + Assert.assertTrue(descriptor.shipFiles.contains(libFile)); + Assert.assertTrue(descriptor.shipFiles.contains(libFolder)); - // only execute part of the deployment to test for shipped files - Set effectiveShipFiles = new HashSet<>(); - descriptor.addLibFolderToShipFiles(effectiveShipFiles); + // only execute part of the deployment to test for shipped files + Set effectiveShipFiles = new HashSet<>(); + descriptor.addLibFolderToShipFiles(effectiveShipFiles); - Assert.assertEquals(0, effectiveShipFiles.size()); - Assert.assertEquals(2, descriptor.shipFiles.size()); - Assert.assertTrue(descriptor.shipFiles.contains(libFile)); - Assert.assertTrue(descriptor.shipFiles.contains(libFolder)); + Assert.assertEquals(0, effectiveShipFiles.size()); + Assert.assertEquals(2, descriptor.shipFiles.size()); + Assert.assertTrue(descriptor.shipFiles.contains(libFile)); + Assert.assertTrue(descriptor.shipFiles.contains(libFolder)); + } finally { + descriptor.close(); + } } /** @@ -83,30 +92,37 @@ public void testExplicitLibShipping() throws Exception { */ @Test public void testEnvironmentLibShipping() throws Exception { - AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(new Configuration(), temporaryFolder.getRoot().getAbsolutePath()); + AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor( + new Configuration(), + temporaryFolder.getRoot().getAbsolutePath(), + YarnClient.createYarnClient()); - File libFolder = temporaryFolder.newFolder().getAbsoluteFile(); - File libFile = new File(libFolder, "libFile.jar"); - libFile.createNewFile(); - - Set effectiveShipFiles = new HashSet<>(); - - final Map oldEnv = System.getenv(); try { - Map env = new HashMap<>(1); - env.put(ConfigConstants.ENV_FLINK_LIB_DIR, libFolder.getAbsolutePath()); - TestBaseUtils.setEnv(env); - // only execute part of the deployment to test for shipped files - descriptor.addLibFolderToShipFiles(effectiveShipFiles); + File libFolder = temporaryFolder.newFolder().getAbsoluteFile(); + File libFile = new File(libFolder, "libFile.jar"); + libFile.createNewFile(); + + Set effectiveShipFiles = new HashSet<>(); + + final Map oldEnv = System.getenv(); + try { + Map env = new HashMap<>(1); + env.put(ConfigConstants.ENV_FLINK_LIB_DIR, libFolder.getAbsolutePath()); + TestBaseUtils.setEnv(env); + // only execute part of the deployment to test for shipped files + descriptor.addLibFolderToShipFiles(effectiveShipFiles); + } finally { + TestBaseUtils.setEnv(oldEnv); + } + + // only add the ship the folder, not the contents + Assert.assertFalse(effectiveShipFiles.contains(libFile)); + Assert.assertTrue(effectiveShipFiles.contains(libFolder)); + Assert.assertFalse(descriptor.shipFiles.contains(libFile)); + Assert.assertFalse(descriptor.shipFiles.contains(libFolder)); } finally { - TestBaseUtils.setEnv(oldEnv); + descriptor.close(); } - - // only add the ship the folder, not the contents - Assert.assertFalse(effectiveShipFiles.contains(libFile)); - Assert.assertTrue(effectiveShipFiles.contains(libFolder)); - Assert.assertFalse(descriptor.shipFiles.contains(libFile)); - Assert.assertFalse(descriptor.shipFiles.contains(libFolder)); } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 86ddd9b4f4b1b..1b677e91280b9 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -22,6 +22,7 @@ import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.IllegalConfigurationException; @@ -35,7 +36,6 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.yarn.configuration.YarnConfigOptions; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsAction; @@ -108,14 +108,9 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor 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 = new YarnConfiguration(); + private final YarnConfiguration yarnConfiguration; - /** - * If the user has specified a different number of slots, we store them here - * Files (usually in a distributed file system) used for the YARN session of Flink. - * Contains configuration files and jar files. - */ - private Path sessionFilesDir; + private final YarnClient yarnClient; private String yarnQueue; @@ -128,7 +123,7 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor /** Lazily initialized list of files to ship. */ protected List shipFiles = new LinkedList<>(); - private final org.apache.flink.configuration.Configuration flinkConfiguration; + private final Configuration flinkConfiguration; private boolean detached; @@ -143,31 +138,48 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor private YarnConfigOptions.UserJarInclusion userJarInclusion; public AbstractYarnClusterDescriptor( - org.apache.flink.configuration.Configuration flinkConfiguration, - String configurationDirectory) { + Configuration flinkConfiguration, + String configurationDirectory, + YarnClient yarnClient) { + + yarnConfiguration = new YarnConfiguration(); + // for unit tests only if (System.getenv("IN_TESTS") != null) { try { - conf.addResource(new File(System.getenv("YARN_CONF_DIR") + "/yarn-site.xml").toURI().toURL()); + yarnConfiguration.addResource(new File(System.getenv("YARN_CONF_DIR") + "/yarn-site.xml").toURI().toURL()); } catch (Throwable t) { throw new RuntimeException("Error", t); } } + this.yarnClient = Preconditions.checkNotNull(yarnClient); + yarnClient.init(yarnConfiguration); + yarnClient.start(); + this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); userJarInclusion = getUserJarInclusionMode(flinkConfiguration); this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory); } + public YarnClient getYarnClient() { + return yarnClient; + } + /** - * The class to bootstrap the application master of the Yarn cluster (runs main method). + * The class to start the application master with. This class runs the main + * method in case of session cluster. */ protected abstract String getYarnSessionClusterEntrypoint(); + /** + * The class to start the application master with. This class runs the main + * method in case of the job cluster. + */ protected abstract String getYarnJobClusterEntrypoint(); - public org.apache.flink.configuration.Configuration getFlinkConfiguration() { + public Configuration getFlinkConfiguration() { return flinkConfiguration; } @@ -257,7 +269,7 @@ private void isReadyForDeployment(ClusterSpecification clusterSpecification) thr // Check if we don't exceed YARN's maximum virtual cores. // The number of cores can be configured in the config. // If not configured, it is set to the number of task slots - int numYarnVcores = conf.getInt(YarnConfiguration.NM_VCORES, YarnConfiguration.DEFAULT_NM_VCORES); + int numYarnVcores = yarnConfiguration.getInt(YarnConfiguration.NM_VCORES, YarnConfiguration.DEFAULT_NM_VCORES); int configuredVcores = flinkConfiguration.getInteger(YarnConfigOptions.VCORES, clusterSpecification.getSlotsPerTaskManager()); // don't configure more than the maximum configured number of vcores if (configuredVcores > numYarnVcores) { @@ -304,21 +316,22 @@ public void setZookeeperNamespace(String zookeeperNamespace) { this.zookeeperNamespace = zookeeperNamespace; } - /** - * Gets a Hadoop Yarn client. - * @return Returns a YarnClient which has to be shutdown manually - */ - public YarnClient getYarnClient() { - YarnClient yarnClient = YarnClient.createYarnClient(); - yarnClient.init(conf); - yarnClient.start(); - return yarnClient; + // ------------------------------------------------------------- + // Lifecycle management + // ------------------------------------------------------------- + + @Override + public void close() { + yarnClient.stop(); } + // ------------------------------------------------------------- + // ClusterClient overrides + // ------------------------------------------------------------- + @Override public YarnClusterClient retrieve(String applicationID) { - YarnClient yarnClient = null; try { // check if required Hadoop environment variables are set. If not, warn user if (System.getenv("HADOOP_CONF_DIR") == null && @@ -329,7 +342,6 @@ public YarnClusterClient retrieve(String applicationID) { } final ApplicationId yarnAppId = ConverterUtils.toApplicationId(applicationID); - yarnClient = getYarnClient(); final ApplicationReport appReport = yarnClient.getApplicationReport(yarnAppId); if (appReport.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) { @@ -349,14 +361,10 @@ public YarnClusterClient retrieve(String applicationID) { this, -1, // we don't know the number of task managers of a started Flink cluster -1, // we don't know how many slots each task manager has for a started Flink cluster - yarnClient, appReport, flinkConfiguration, false); } catch (Exception e) { - if (null != yarnClient) { - yarnClient.stop(); - } throw new RuntimeException("Couldn't retrieve Yarn cluster", e); } } @@ -414,8 +422,6 @@ protected YarnClusterClient deployInternal( isReadyForDeployment(clusterSpecification); - final YarnClient yarnClient = getYarnClient(); - // ------------------ Check if the specified queue exists -------------------- checkYarnQueues(yarnClient); @@ -442,7 +448,7 @@ protected YarnClusterClient deployInternal( throw new YarnDeploymentException("Could not retrieve information about free cluster resources.", e); } - final int yarnMinAllocationMB = conf.getInt("yarn.scheduler.minimum-allocation-mb", 0); + final int yarnMinAllocationMB = yarnConfiguration.getInt("yarn.scheduler.minimum-allocation-mb", 0); final ClusterSpecification validClusterSpecification; try { @@ -477,7 +483,6 @@ protected YarnClusterClient deployInternal( this, clusterSpecification.getNumberTaskManagers(), clusterSpecification.getSlotsPerTaskManager(), - yarnClient, report, flinkConfiguration, true); @@ -627,7 +632,7 @@ public ApplicationReport startAppMaster( // initialize file system // Copy the application master jar to the filesystem // Create a local resource to point to the destination jar path - final FileSystem fs = FileSystem.get(conf); + final FileSystem fs = FileSystem.get(yarnConfiguration); final Path homeDir = fs.getHomeDirectory(); // hard coded check for the GoogleHDFS client because its not overriding the getScheme() method. @@ -881,7 +886,7 @@ public ApplicationReport startAppMaster( if (UserGroupInformation.isSecurityEnabled()) { // set HDFS delegation tokens when security is enabled LOG.info("Adding delegation token to the AM container.."); - Utils.setTokensFor(amContainer, paths, conf); + Utils.setTokensFor(amContainer, paths, yarnConfiguration); } amContainer.setLocalResources(localResources); @@ -926,7 +931,7 @@ public ApplicationReport startAppMaster( } // set classpath from YARN configuration - Utils.setupYarnClassPath(conf, appMasterEnv); + Utils.setupYarnClassPath(yarnConfiguration, appMasterEnv); amContainer.setEnvironment(appMasterEnv); @@ -1196,7 +1201,6 @@ public String getClusterDescription() { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintStream ps = new PrintStream(baos); - YarnClient yarnClient = getYarnClient(); YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers()); @@ -1223,7 +1227,6 @@ public String getClusterDescription() { ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " + q.getMaximumCapacity() + " Applications: " + q.getApplications().size()); } - yarnClient.stop(); return baos.toString(); } catch (Exception e) { throw new RuntimeException("Couldn't get cluster description", e); @@ -1411,7 +1414,7 @@ public void run() { failSessionDuringDeployment(yarnClient, yarnApplication); LOG.info("Deleting files in {}.", yarnFilesDir); try { - FileSystem fs = FileSystem.get(conf); + FileSystem fs = FileSystem.get(yarnConfiguration); if (!fs.delete(yarnFilesDir, true)) { throw new IOException("Deleting files in " + yarnFilesDir + " was unsuccessful"); @@ -1419,7 +1422,7 @@ public void run() { fs.close(); } catch (IOException e) { - LOG.error("Failed to delete Flink Jar and conf files in HDFS", e); + LOG.error("Failed to delete Flink Jar and configuration files in HDFS", e); } } } @@ -1525,7 +1528,6 @@ protected YarnClusterClient createYarnClusterClient( AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, - YarnClient yarnClient, ApplicationReport report, org.apache.flink.configuration.Configuration flinkConfiguration, boolean perJobCluster) throws Exception { @@ -1533,7 +1535,6 @@ protected YarnClusterClient createYarnClusterClient( descriptor, numberTaskManagers, slotsPerTaskManager, - yarnClient, report, flinkConfiguration, perJobCluster); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java index 80d09436ed6b1..63421f922d0e8 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java @@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.exceptions.YarnException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,8 +66,6 @@ public class YarnClusterClient extends ClusterClient { private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClient.class); - private YarnClient yarnClient; - private Thread clientShutdownHook = new ClientShutdownHook(); //---------- Class internal fields ------------------- @@ -93,7 +90,6 @@ public class YarnClusterClient extends ClusterClient { * @param clusterDescriptor The descriptor used at cluster creation * @param numberTaskManagers The number of task managers, -1 if unknown * @param slotsPerTaskManager Slots per task manager, -1 if unknown - * @param yarnClient Client to talk to YARN * @param appReport the YARN application ID * @param flinkConfig Flink configuration * @param newlyCreatedCluster Indicator whether this cluster has just been created @@ -104,7 +100,6 @@ public YarnClusterClient( final AbstractYarnClusterDescriptor clusterDescriptor, final int numberTaskManagers, final int slotsPerTaskManager, - final YarnClient yarnClient, final ApplicationReport appReport, Configuration flinkConfig, boolean newlyCreatedCluster) throws Exception { @@ -115,7 +110,6 @@ public YarnClusterClient( this.clusterDescriptor = clusterDescriptor; this.numberTaskManagers = numberTaskManagers; this.slotsPerTaskManager = slotsPerTaskManager; - this.yarnClient = yarnClient; this.appReport = appReport; this.appId = appReport.getApplicationId(); this.trackingURL = appReport.getTrackingUrl(); @@ -328,7 +322,7 @@ public void shutdownCluster() { Future response = Patterns.ask(applicationClient.get(), new YarnMessages.LocalStopYarnSession(ApplicationStatus.CANCELED, - "Flink YARN Client requested shutdown"), + "Flink YARN Client requested shutdown"), new Timeout(akkaDuration)); Await.ready(response, akkaDuration); } catch (Exception e) { @@ -349,7 +343,7 @@ public void shutdownCluster() { } try { - ApplicationReport appReport = yarnClient.getApplicationReport(appId); + ApplicationReport appReport = clusterDescriptor.getYarnClient().getApplicationReport(appId); LOG.info("Application " + appId + " finished with state " + appReport .getYarnApplicationState() + " and final state " + appReport @@ -368,10 +362,6 @@ public void shutdownCluster() { } catch (Exception e) { LOG.warn("Couldn't get final report", e); } - - LOG.info("YARN Client is shutting down"); - yarnClient.stop(); // actorRunner is using the yarnClient. - yarnClient = null; // set null to clearly see if somebody wants to access it afterwards. } public boolean hasBeenShutdown() { diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 8759c3e9144be..76f9154e5879b 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -22,13 +22,18 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.hadoop.yarn.client.api.YarnClient; + /** * Default implementation of {@link AbstractYarnClusterDescriptor} which starts an {@link YarnApplicationMasterRunner}. */ public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor { - public YarnClusterDescriptor(Configuration flinkConfiguration, String configurationDirectory) { - super(flinkConfiguration, configurationDirectory); + public YarnClusterDescriptor( + Configuration flinkConfiguration, + String configurationDirectory, + YarnClient yarnClient) { + super(flinkConfiguration, configurationDirectory, yarnClient); } @Override diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java index ed04523a8287d..6ce192c82afaa 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java @@ -22,14 +22,19 @@ import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; +import org.apache.hadoop.yarn.client.api.YarnClient; + /** * Implementation of {@link org.apache.flink.yarn.AbstractYarnClusterDescriptor} which is used to start the * new application master for a job under flip-6. */ public class YarnClusterDescriptorV2 extends AbstractYarnClusterDescriptor { - public YarnClusterDescriptorV2(Configuration flinkConfiguration, String configurationDirectory) { - super(flinkConfiguration, configurationDirectory); + public YarnClusterDescriptorV2( + Configuration flinkConfiguration, + String configurationDirectory, + YarnClient yarnCLient) { + super(flinkConfiguration, configurationDirectory, yarnCLient); } @Override diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 54837585eaeb8..c045082d73f65 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -50,6 +50,7 @@ import org.apache.commons.cli.PosixParser; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.util.ConverterUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -862,10 +863,11 @@ public static File getYarnPropertiesLocation(Configuration conf) { } protected AbstractYarnClusterDescriptor getClusterDescriptor(Configuration configuration, String configurationDirectory, boolean flip6) { + final YarnClient yarnClient = YarnClient.createYarnClient(); if (flip6) { - return new YarnClusterDescriptorV2(configuration, configurationDirectory); + return new YarnClusterDescriptorV2(configuration, configurationDirectory, yarnClient); } else { - return new YarnClusterDescriptor(configuration, configurationDirectory); + return new YarnClusterDescriptor(configuration, configurationDirectory, yarnClient); } } } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java index c11c4139a11c3..0d1bf6586aecc 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.client.api.YarnClient; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -59,9 +60,12 @@ public void beforeTest() throws IOException { @Test public void testFailIfTaskSlotsHigherThanMaxVcores() { + final YarnClient yarnClient = YarnClient.createYarnClient(); + YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( new Configuration(), - temporaryFolder.getRoot().getAbsolutePath()); + temporaryFolder.getRoot().getAbsolutePath(), + yarnClient); clusterDescriptor.setLocalJarPath(new Path(flinkJar.getPath())); @@ -81,6 +85,8 @@ public void testFailIfTaskSlotsHigherThanMaxVcores() { if (!(e.getCause() instanceof IllegalConfigurationException)) { throw e; } + } finally { + clusterDescriptor.close(); } } @@ -90,9 +96,12 @@ public void testConfigOverwrite() { // overwrite vcores in config configuration.setInteger(YarnConfigOptions.VCORES, Integer.MAX_VALUE); + final YarnClient yarnClient = YarnClient.createYarnClient(); + YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( configuration, - temporaryFolder.getRoot().getAbsolutePath()); + temporaryFolder.getRoot().getAbsolutePath(), + yarnClient); clusterDescriptor.setLocalJarPath(new Path(flinkJar.getPath())); @@ -113,15 +122,19 @@ public void testConfigOverwrite() { if (!(e.getCause() instanceof IllegalConfigurationException)) { throw e; } + } finally { + clusterDescriptor.close(); } } @Test public void testSetupApplicationMasterContainer() { Configuration cfg = new Configuration(); + final YarnClient yarnClient = YarnClient.createYarnClient(); YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( cfg, - temporaryFolder.getRoot().getAbsolutePath()); + temporaryFolder.getRoot().getAbsolutePath(), + yarnClient); final String java = "$JAVA_HOME/bin/java"; final String jvmmem = "-Xmx424m"; @@ -142,219 +155,223 @@ public void testSetupApplicationMasterContainer() { "2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err"; final int jobManagerMemory = 1024; - // no logging, with/out krb5 - assertEquals( - java + " " + jvmmem + - " " + // jvmOpts - " " + // logging - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - false, - false, - false, - jobManagerMemory) - .getCommands().get(0)); - - assertEquals( - java + " " + jvmmem + - " " + " " + krb5 + // jvmOpts - " " + // logging - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - false, - false, - true, - jobManagerMemory) - .getCommands().get(0)); - - // logback only, with/out krb5 - assertEquals( - java + " " + jvmmem + - " " + // jvmOpts - " " + logfile + " " + logback + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - false, - false, - jobManagerMemory) - .getCommands().get(0)); - - assertEquals( - java + " " + jvmmem + - " " + " " + krb5 + // jvmOpts - " " + logfile + " " + logback + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - false, - true, - jobManagerMemory) - .getCommands().get(0)); - - // log4j, with/out krb5 - assertEquals( - java + " " + jvmmem + - " " + // jvmOpts - " " + logfile + " " + log4j + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - false, - true, - false, - jobManagerMemory) - .getCommands().get(0)); - - assertEquals( - java + " " + jvmmem + - " " + " " + krb5 + // jvmOpts - " " + logfile + " " + log4j + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - false, - true, - true, - jobManagerMemory) - .getCommands().get(0)); - - // logback + log4j, with/out krb5 - assertEquals( - java + " " + jvmmem + - " " + // jvmOpts - " " + logfile + " " + logback + " " + log4j + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - true, - false, - jobManagerMemory) - .getCommands().get(0)); - - assertEquals( - java + " " + jvmmem + - " " + " " + krb5 + // jvmOpts - " " + logfile + " " + logback + " " + log4j + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - true, - true, - jobManagerMemory) - .getCommands().get(0)); - - // logback + log4j, with/out krb5, different JVM opts - // IMPORTANT: Be aware that we are using side effects here to modify the created YarnClusterDescriptor, - // because we have a reference to the ClusterDescriptor's configuration which we modify continuously - cfg.setString(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); - assertEquals( - java + " " + jvmmem + - " " + jvmOpts + - " " + logfile + " " + logback + " " + log4j + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - true, - false, - jobManagerMemory) - .getCommands().get(0)); - - assertEquals( - java + " " + jvmmem + - " " + jvmOpts + " " + krb5 + // jvmOpts - " " + logfile + " " + logback + " " + log4j + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - true, - true, - jobManagerMemory) - .getCommands().get(0)); - - // logback + log4j, with/out krb5, different JVM opts - // IMPORTANT: Be aware that we are using side effects here to modify the created YarnClusterDescriptor - cfg.setString(CoreOptions.FLINK_JM_JVM_OPTIONS, jmJvmOpts); - assertEquals( - java + " " + jvmmem + - " " + jvmOpts + " " + jmJvmOpts + - " " + logfile + " " + logback + " " + log4j + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - true, - false, - jobManagerMemory) - .getCommands().get(0)); - - assertEquals( - java + " " + jvmmem + - " " + jvmOpts + " " + jmJvmOpts + " " + krb5 + // jvmOpts - " " + logfile + " " + logback + " " + log4j + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - true, - true, - jobManagerMemory) - .getCommands().get(0)); - - // now try some configurations with different yarn.container-start-command-template - // IMPORTANT: Be aware that we are using side effects here to modify the created YarnClusterDescriptor - cfg.setString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, - "%java% 1 %jvmmem% 2 %jvmopts% 3 %logging% 4 %class% 5 %args% 6 %redirects%"); - assertEquals( - java + " 1 " + jvmmem + - " 2 " + jvmOpts + " " + jmJvmOpts + " " + krb5 + // jvmOpts - " 3 " + logfile + " " + logback + " " + log4j + - " 4 " + mainClass + " 5 " + args + " 6 " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - true, - true, - jobManagerMemory) - .getCommands().get(0)); - - cfg.setString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, - "%java% %logging% %jvmopts% %jvmmem% %class% %args% %redirects%"); - // IMPORTANT: Be aware that we are using side effects here to modify the created YarnClusterDescriptor - assertEquals( - java + - " " + logfile + " " + logback + " " + log4j + - " " + jvmOpts + " " + jmJvmOpts + " " + krb5 + // jvmOpts - " " + jvmmem + - " " + mainClass + " " + args + " " + redirects, - clusterDescriptor - .setupApplicationMasterContainer( - mainClass, - true, - true, - true, - jobManagerMemory) - .getCommands().get(0)); + try { + // no logging, with/out krb5 + assertEquals( + java + " " + jvmmem + + " " + // jvmOpts + " " + // logging + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + false, + false, + false, + jobManagerMemory) + .getCommands().get(0)); + + assertEquals( + java + " " + jvmmem + + " " + " " + krb5 + // jvmOpts + " " + // logging + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + false, + false, + true, + jobManagerMemory) + .getCommands().get(0)); + + // logback only, with/out krb5 + assertEquals( + java + " " + jvmmem + + " " + // jvmOpts + " " + logfile + " " + logback + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + false, + false, + jobManagerMemory) + .getCommands().get(0)); + + assertEquals( + java + " " + jvmmem + + " " + " " + krb5 + // jvmOpts + " " + logfile + " " + logback + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + false, + true, + jobManagerMemory) + .getCommands().get(0)); + + // log4j, with/out krb5 + assertEquals( + java + " " + jvmmem + + " " + // jvmOpts + " " + logfile + " " + log4j + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + false, + true, + false, + jobManagerMemory) + .getCommands().get(0)); + + assertEquals( + java + " " + jvmmem + + " " + " " + krb5 + // jvmOpts + " " + logfile + " " + log4j + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + false, + true, + true, + jobManagerMemory) + .getCommands().get(0)); + + // logback + log4j, with/out krb5 + assertEquals( + java + " " + jvmmem + + " " + // jvmOpts + " " + logfile + " " + logback + " " + log4j + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + true, + false, + jobManagerMemory) + .getCommands().get(0)); + + assertEquals( + java + " " + jvmmem + + " " + " " + krb5 + // jvmOpts + " " + logfile + " " + logback + " " + log4j + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + true, + true, + jobManagerMemory) + .getCommands().get(0)); + + // logback + log4j, with/out krb5, different JVM opts + // IMPORTANT: Be aware that we are using side effects here to modify the created YarnClusterDescriptor, + // because we have a reference to the ClusterDescriptor's configuration which we modify continuously + cfg.setString(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); + assertEquals( + java + " " + jvmmem + + " " + jvmOpts + + " " + logfile + " " + logback + " " + log4j + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + true, + false, + jobManagerMemory) + .getCommands().get(0)); + + assertEquals( + java + " " + jvmmem + + " " + jvmOpts + " " + krb5 + // jvmOpts + " " + logfile + " " + logback + " " + log4j + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + true, + true, + jobManagerMemory) + .getCommands().get(0)); + + // logback + log4j, with/out krb5, different JVM opts + // IMPORTANT: Be aware that we are using side effects here to modify the created YarnClusterDescriptor + cfg.setString(CoreOptions.FLINK_JM_JVM_OPTIONS, jmJvmOpts); + assertEquals( + java + " " + jvmmem + + " " + jvmOpts + " " + jmJvmOpts + + " " + logfile + " " + logback + " " + log4j + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + true, + false, + jobManagerMemory) + .getCommands().get(0)); + + assertEquals( + java + " " + jvmmem + + " " + jvmOpts + " " + jmJvmOpts + " " + krb5 + // jvmOpts + " " + logfile + " " + logback + " " + log4j + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + true, + true, + jobManagerMemory) + .getCommands().get(0)); + + // now try some configurations with different yarn.container-start-command-template + // IMPORTANT: Be aware that we are using side effects here to modify the created YarnClusterDescriptor + cfg.setString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, + "%java% 1 %jvmmem% 2 %jvmopts% 3 %logging% 4 %class% 5 %args% 6 %redirects%"); + assertEquals( + java + " 1 " + jvmmem + + " 2 " + jvmOpts + " " + jmJvmOpts + " " + krb5 + // jvmOpts + " 3 " + logfile + " " + logback + " " + log4j + + " 4 " + mainClass + " 5 " + args + " 6 " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + true, + true, + jobManagerMemory) + .getCommands().get(0)); + + cfg.setString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, + "%java% %logging% %jvmopts% %jvmmem% %class% %args% %redirects%"); + // IMPORTANT: Be aware that we are using side effects here to modify the created YarnClusterDescriptor + assertEquals( + java + + " " + logfile + " " + logback + " " + log4j + + " " + jvmOpts + " " + jmJvmOpts + " " + krb5 + // jvmOpts + " " + jvmmem + + " " + mainClass + " " + args + " " + redirects, + clusterDescriptor + .setupApplicationMasterContainer( + mainClass, + true, + true, + true, + jobManagerMemory) + .getCommands().get(0)); + } finally { + clusterDescriptor.close(); + } } } From cf9e0e315c08dc1a39a38f38687c6440ad02f0e2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 10 Jan 2018 15:00:59 +0100 Subject: [PATCH 05/12] fixup! [FLINK-8329] [flip6] Move YarnClient to AbstractYarnClusterDescriptor --- .../org/apache/flink/yarn/YARNITCase.java | 43 +++++----- .../flink/yarn/YARNSessionFIFOITCase.java | 81 ++++++++++--------- .../yarn/AbstractYarnClusterDescriptor.java | 2 +- 3 files changed, 64 insertions(+), 62 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java index d9b468dee31ca..069f68a019660 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java @@ -54,38 +54,39 @@ public void testPerJobMode() throws Exception { Configuration configuration = new Configuration(); configuration.setString(AkkaOptions.ASK_TIMEOUT, "30 s"); final YarnClient yarnClient = YarnClient.createYarnClient(); - YarnClusterDescriptorV2 yarnClusterDescriptorV2 = new YarnClusterDescriptorV2( + + try (final YarnClusterDescriptorV2 yarnClusterDescriptorV2 = new YarnClusterDescriptorV2( configuration, System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR), - yarnClient); + yarnClient)) { - yarnClusterDescriptorV2.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); - yarnClusterDescriptorV2.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); + yarnClusterDescriptorV2.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); + yarnClusterDescriptorV2.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); - final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(768) - .setTaskManagerMemoryMB(1024) - .setSlotsPerTaskManager(1) - .setNumberTaskManagers(1) - .createClusterSpecification(); + final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(768) + .setTaskManagerMemoryMB(1024) + .setSlotsPerTaskManager(1) + .setNumberTaskManagers(1) + .createClusterSpecification(); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); - env.addSource(new InfiniteSource()) - .shuffle() - .addSink(new DiscardingSink()); + env.addSource(new InfiniteSource()) + .shuffle() + .addSink(new DiscardingSink()); - final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - File testingJar = YarnTestBase.findFile("..", new TestingYarnClusterDescriptor.TestJarFinder("flink-yarn-tests")); + File testingJar = YarnTestBase.findFile("..", new TestingYarnClusterDescriptor.TestJarFinder("flink-yarn-tests")); - jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI())); + jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI())); - YarnClusterClient clusterClient = yarnClusterDescriptorV2.deployJobCluster(clusterSpecification, jobGraph); + YarnClusterClient clusterClient = yarnClusterDescriptorV2.deployJobCluster(clusterSpecification, jobGraph); - clusterClient.shutdown(); - yarnClusterDescriptorV2.close(); + clusterClient.shutdown(); + } } private static class InfiniteSource implements ParallelSourceFunction { 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 fe44a15cdfcac..ec8ef501480e7 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 @@ -228,54 +228,55 @@ public void testJavaAPI() throws Exception { String confDirPath = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); Configuration configuration = GlobalConfiguration.loadConfiguration(); final YarnClient yarnClient = YarnClient.createYarnClient(); - AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( + + try (final AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( configuration, confDirPath, - yarnClient); - Assert.assertNotNull("unable to get yarn client", clusterDescriptor); - clusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); - clusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); + yarnClient)) { + Assert.assertNotNull("unable to get yarn client", clusterDescriptor); + clusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); + clusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); - final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(768) - .setTaskManagerMemoryMB(1024) - .setNumberTaskManagers(1) - .setSlotsPerTaskManager(1) - .createClusterSpecification(); - // deploy - ClusterClient yarnCluster = null; - try { - yarnCluster = clusterDescriptor.deploySessionCluster(clusterSpecification); - } catch (Exception e) { - LOG.warn("Failing test", e); - Assert.fail("Error while deploying YARN cluster: " + e.getMessage()); - } - GetClusterStatusResponse expectedStatus = new GetClusterStatusResponse(1, 1); - for (int second = 0; second < waitTime * 2; second++) { // run "forever" + final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(768) + .setTaskManagerMemoryMB(1024) + .setNumberTaskManagers(1) + .setSlotsPerTaskManager(1) + .createClusterSpecification(); + // deploy + ClusterClient yarnCluster = null; try { - Thread.sleep(1000); - } catch (InterruptedException e) { - LOG.warn("Interrupted", e); - } - GetClusterStatusResponse status = yarnCluster.getClusterStatus(); - if (status != null && status.equals(expectedStatus)) { - LOG.info("ClusterClient reached status " + status); - break; // all good, cluster started + yarnCluster = clusterDescriptor.deploySessionCluster(clusterSpecification); + } catch (Exception e) { + LOG.warn("Failing test", e); + Assert.fail("Error while deploying YARN cluster: " + e.getMessage()); } - if (second > waitTime) { - // we waited for 15 seconds. cluster didn't come up correctly - Assert.fail("The custer didn't start after " + waitTime + " seconds"); + GetClusterStatusResponse expectedStatus = new GetClusterStatusResponse(1, 1); + for (int second = 0; second < waitTime * 2; second++) { // run "forever" + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + LOG.warn("Interrupted", e); + } + GetClusterStatusResponse status = yarnCluster.getClusterStatus(); + if (status != null && status.equals(expectedStatus)) { + LOG.info("ClusterClient reached status " + status); + break; // all good, cluster started + } + if (second > waitTime) { + // we waited for 15 seconds. cluster didn't come up correctly + Assert.fail("The custer didn't start after " + waitTime + " seconds"); + } } - } - // use the cluster - Assert.assertNotNull(yarnCluster.getJobManagerAddress()); - Assert.assertNotNull(yarnCluster.getWebInterfaceURL()); + // use the cluster + Assert.assertNotNull(yarnCluster.getJobManagerAddress()); + Assert.assertNotNull(yarnCluster.getWebInterfaceURL()); - LOG.info("Shutting down cluster. All tests passed"); - // shutdown cluster - yarnCluster.shutdown(); - clusterDescriptor.close(); + LOG.info("Shutting down cluster. All tests passed"); + // shutdown cluster + yarnCluster.shutdown(); + } LOG.info("Finished testJavaAPI()"); } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 1b677e91280b9..03723196ee620 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -147,7 +147,7 @@ public AbstractYarnClusterDescriptor( // for unit tests only if (System.getenv("IN_TESTS") != null) { try { - yarnConfiguration.addResource(new File(System.getenv("YARN_CONF_DIR") + "/yarn-site.xml").toURI().toURL()); + yarnConfiguration.addResource(new File(System.getenv("YARN_CONF_DIR"), "yarn-site.xml").toURI().toURL()); } catch (Throwable t) { throw new RuntimeException("Error", t); } From b465d8609e8adb26d3f5c1a5d621efcfe47271bc Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 18 Dec 2017 18:59:30 +0100 Subject: [PATCH 06/12] [FLINK-8332] [flip6] Move savepoint dispose into ClusterClient Move the savepoint disposal logic from the CliFrontend into the ClusterClient. This gives a better separation of concerns and allows the CliFrontend to be used with different ClusterClient implementations. --- .../org/apache/flink/client/CliFrontend.java | 192 +++++-------- .../flink/client/program/ClusterClient.java | 35 +++ .../client/CliFrontendSavepointTest.java | 266 +++++++----------- .../flink/client/util/MockedCliFrontend.java | 5 + 4 files changed, 219 insertions(+), 279 deletions(-) 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 dff12f6b70d52..70dacf3735ba8 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 @@ -57,9 +57,9 @@ 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.instance.ActorGateway; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.runtime.util.EnvironmentInformation; @@ -91,14 +91,10 @@ import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; -import scala.concurrent.Await; -import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; -import static org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint; -import static org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepointFailure; - /** * Implementation of a simple command line frontend for executing programs. */ @@ -659,39 +655,53 @@ protected int savepoint(String[] args) { return 0; } - if (options.isDispose()) { - // Discard - return disposeSavepoint(options); - } else { - // Trigger - String[] cleanedArgs = options.getArgs(); - JobID jobId; + CustomCommandLine customCommandLine = getActiveCustomCommandLine(options.getCommandLine()); - if (cleanedArgs.length >= 1) { - String jobIdString = cleanedArgs[0]; - try { - jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); - } catch (Exception e) { - return handleArgException(new IllegalArgumentException( - "Error: The value for the Job ID is not a valid ID.")); - } + ClusterClient clusterClient = customCommandLine.retrieveCluster(options.getCommandLine(), config, configurationDirectory); + + try { + if (options.isDispose()) { + // Discard + return disposeSavepoint(clusterClient, options.getSavepointPath()); } else { - return handleArgException(new IllegalArgumentException( + // Trigger + String[] cleanedArgs = options.getArgs(); + JobID jobId; + + if (cleanedArgs.length >= 1) { + String jobIdString = cleanedArgs[0]; + try { + jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); + } catch (Exception e) { + return handleArgException(new IllegalArgumentException( + "Error: The value for the Job ID is not a valid ID.")); + } + } else { + return handleArgException(new IllegalArgumentException( "Error: The value for the Job ID is not a valid ID. " + - "Specify a Job ID to trigger a savepoint.")); - } + "Specify a Job ID to trigger a savepoint.")); + } - String savepointDirectory = null; - if (cleanedArgs.length >= 2) { - savepointDirectory = cleanedArgs[1]; - } + String savepointDirectory = null; + if (cleanedArgs.length >= 2) { + savepointDirectory = cleanedArgs[1]; + } - // Print superfluous arguments - if (cleanedArgs.length >= 3) { - logAndSysout("Provided more arguments than required. Ignoring not needed arguments."); - } + // Print superfluous arguments + if (cleanedArgs.length >= 3) { + logAndSysout("Provided more arguments than required. Ignoring not needed arguments."); + } - return triggerSavepoint(options, jobId, savepointDirectory); + return triggerSavepoint(clusterClient, jobId, savepointDirectory); + } + } catch (Exception e) { + return handleError(e); + } finally { + try { + clusterClient.shutdown(); + } catch (Exception e) { + LOG.info("Could not shutdown the cluster client.", e); + } } } @@ -699,88 +709,53 @@ protected int savepoint(String[] args) { * Sends a {@link org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint} * message to the job manager. */ - private int triggerSavepoint(SavepointOptions options, JobID jobId, String savepointDirectory) { - try { - CustomCommandLine activeCommandLine = getActiveCustomCommandLine(options.getCommandLine()); - ClusterClient client = activeCommandLine.retrieveCluster(options.getCommandLine(), config, configurationDirectory); - try { - logAndSysout("Triggering savepoint for job " + jobId + "."); - CompletableFuture savepointPathFuture = client.triggerSavepoint(jobId, savepointDirectory); - - String savepointPath; - try { - logAndSysout("Waiting for response..."); - savepointPath = savepointPathFuture.get(); - } - catch (ExecutionException ee) { - Throwable cause = ExceptionUtils.stripExecutionException(ee); - throw new FlinkException("Triggering a savepoint for the job " + jobId + " failed.", cause); - } - - logAndSysout("Savepoint completed. Path: " + savepointPath); - logAndSysout("You can resume your program from this savepoint with the run command."); + private int triggerSavepoint(ClusterClient clusterClient, JobID jobId, String savepointDirectory) throws Exception { + logAndSysout("Triggering savepoint for job " + jobId + "."); + CompletableFuture savepointPathFuture = clusterClient.triggerSavepoint(jobId, savepointDirectory); - return 0; - } - finally { - client.shutdown(); - } + String savepointPath; + try { + logAndSysout("Waiting for response..."); + savepointPath = savepointPathFuture.get(); } - catch (Throwable t) { - return handleError(t); + catch (ExecutionException ee) { + Throwable cause = ExceptionUtils.stripExecutionException(ee); + throw new FlinkException("Triggering a savepoint for the job " + jobId + " failed.", cause); } + + logAndSysout("Savepoint completed. Path: " + savepointPath); + logAndSysout("You can resume your program from this savepoint with the run command."); + + return 0; } /** * Sends a {@link org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint} * message to the job manager. */ - private int disposeSavepoint(SavepointOptions options) { - try { - String savepointPath = options.getSavepointPath(); - if (savepointPath == null) { - throw new IllegalArgumentException("Missing required argument: savepoint path. " + - "Usage: bin/flink savepoint -d "); - } - - ActorGateway jobManager = getJobManagerGateway(options); + private int disposeSavepoint(ClusterClient clusterClient, String savepointPath) { + Preconditions.checkNotNull(savepointPath, "Missing required argument: savepoint path. " + + "Usage: bin/flink savepoint -d "); - logAndSysout("Disposing savepoint '" + savepointPath + "'."); + logAndSysout("Disposing savepoint '" + savepointPath + "'."); - Object msg = new DisposeSavepoint(savepointPath); - Future response = jobManager.ask(msg, clientTimeout); + CompletableFuture disposeFuture = null; + try { + disposeFuture = clusterClient.disposeSavepoint(savepointPath, FutureUtils.toTime(clientTimeout)); + } catch (Exception e) { + return handleError(new FlinkException("Could not dispose savepoint", e)); + } - Object result; - try { - logAndSysout("Waiting for response..."); - result = Await.result(response, clientTimeout); - } catch (Exception e) { - throw new Exception("Disposing the savepoint with path" + savepointPath + " failed.", e); - } + logAndSysout("Waiting for response..."); - if (result.getClass() == JobManagerMessages.getDisposeSavepointSuccess().getClass()) { - logAndSysout("Savepoint '" + savepointPath + "' disposed."); - return 0; - } else if (result instanceof DisposeSavepointFailure) { - DisposeSavepointFailure failure = (DisposeSavepointFailure) result; - - if (failure.cause() instanceof ClassNotFoundException) { - throw new ClassNotFoundException("Savepoint disposal failed, because of a " + - "missing class. This is most likely caused by a custom state " + - "instance, which cannot be disposed without the user code class " + - "loader. Please provide the program jar with which you have created " + - "the savepoint via -j for disposal.", - failure.cause().getCause()); - } else { - throw failure.cause(); - } - } else { - throw new IllegalStateException("Unknown JobManager response of type " + - result.getClass()); - } - } catch (Throwable t) { - return handleError(t); + try { + disposeFuture.get(clientTimeout.toMillis(), TimeUnit.MILLISECONDS); + } catch (Exception e) { + return handleError(e); } + + logAndSysout("Savepoint '" + savepointPath + "' disposed."); + return 0; } // -------------------------------------------------------------------------------------------- @@ -882,19 +857,6 @@ protected ClusterClient retrieveClient(CommandLineOptions options) { } } - /** - * Retrieves the {@link ActorGateway} for the JobManager. The ClusterClient is retrieved - * from the provided {@link CommandLineOptions}. - * - * @param options CommandLineOptions specifying the JobManager URL - * @return Gateway to the JobManager - * @throws Exception - */ - protected ActorGateway getJobManagerGateway(CommandLineOptions options) throws Exception { - logAndSysout("Retrieving JobManager."); - return retrieveClient(options).getJobManagerGateway(); - } - /** * Creates a {@link ClusterClient} object from the given command line options and other parameters. * @param options Command line options diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index 3515363b3dfd1..b91927aba1e49 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -52,6 +52,7 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsErroneous; import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsFound; @@ -62,6 +63,7 @@ import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -691,6 +693,39 @@ public CompletableFuture triggerSavepoint(JobID jobId, @Nullable String }); } + public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) throws Exception { + final ActorGateway jobManager = getJobManagerGateway(); + + Object msg = new JobManagerMessages.DisposeSavepoint(savepointPath); + CompletableFuture responseFuture = FutureUtils.toJava( + jobManager.ask( + msg, + FutureUtils.toFiniteDuration(timeout))); + + return responseFuture.thenApply( + (Object response) -> { + if (response instanceof JobManagerMessages.DisposeSavepointSuccess$) { + return Acknowledge.get(); + } else if (response instanceof JobManagerMessages.DisposeSavepointFailure) { + JobManagerMessages.DisposeSavepointFailure failureResponse = (JobManagerMessages.DisposeSavepointFailure) response; + + if (failureResponse.cause() instanceof ClassNotFoundException) { + throw new CompletionException( + new ClassNotFoundException("Savepoint disposal failed, because of a " + + "missing class. This is most likely caused by a custom state " + + "instance, which cannot be disposed without the user code class " + + "loader. Please provide the program jar with which you have created " + + "the savepoint via -j for disposal.", + failureResponse.cause().getCause())); + } else { + throw new CompletionException(failureResponse.cause()); + } + } else { + throw new CompletionException(new FlinkRuntimeException("Unknown response type " + response.getClass().getSimpleName() + '.')); + } + }); + } + /** * Lists the currently running and finished jobs on the cluster. * diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java index 1f0d3562023ce..8a81e1b066dab 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java @@ -19,32 +19,29 @@ package org.apache.flink.client; import org.apache.flink.api.common.JobID; -import org.apache.flink.client.cli.CommandLineOptions; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.client.util.MockedCliFrontend; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLogger; -import akka.dispatch.Futures; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.mockito.Mockito; import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.PrintStream; import java.util.concurrent.CompletableFuture; +import java.util.function.BiFunction; import java.util.zip.ZipOutputStream; -import scala.concurrent.Future; -import scala.concurrent.Promise; -import scala.concurrent.duration.FiniteDuration; - -import static org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint; -import static org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepointFailure; -import static org.apache.flink.runtime.messages.JobManagerMessages.getDisposeSavepointSuccess; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; @@ -60,7 +57,7 @@ /** * Tests for the SAVEPOINT command. */ -public class CliFrontendSavepointTest { +public class CliFrontendSavepointTest extends TestLogger { private static PrintStream stdOut; private static PrintStream stdErr; @@ -77,23 +74,25 @@ public class CliFrontendSavepointTest { public void testTriggerSavepointSuccess() throws Exception { replaceStdOutAndStdErr(); - try { - JobID jobId = new JobID(); + JobID jobId = new JobID(); - String savepointPath = "expectedSavepointPath"; + String savepointPath = "expectedSavepointPath"; - MockedCliFrontend frontend = new SavepointTestCliFrontend(savepointPath); + final ClusterClient clusterClient = createClusterClient(savepointPath); + + try { + MockedCliFrontend frontend = new MockedCliFrontend(clusterClient); String[] parameters = { jobId.toString() }; - int returnCode = frontend.savepoint(parameters); + frontend.savepoint(parameters); - assertEquals(0, returnCode); - verify(frontend.client, times(1)) + verify(clusterClient, times(1)) .triggerSavepoint(eq(jobId), isNull(String.class)); assertTrue(buffer.toString().contains(savepointPath)); } finally { + clusterClient.shutdown(); restoreStdOutAndStdErr(); } } @@ -102,23 +101,27 @@ public void testTriggerSavepointSuccess() throws Exception { public void testTriggerSavepointFailure() throws Exception { replaceStdOutAndStdErr(); - try { - JobID jobId = new JobID(); + JobID jobId = new JobID(); + + String expectedTestException = "expectedTestException"; + Exception testException = new Exception(expectedTestException); - Exception testException = new Exception("expectedTestException"); + final ClusterClient clusterClient = createFailingClusterClient(testException); - MockedCliFrontend frontend = new SavepointTestCliFrontend(testException); + try { + MockedCliFrontend frontend = new MockedCliFrontend(clusterClient); String[] parameters = { jobId.toString() }; + int returnCode = frontend.savepoint(parameters); assertNotEquals(0, returnCode); - verify(frontend.client, times(1)) - .triggerSavepoint(eq(jobId), isNull(String.class)); - assertTrue(buffer.toString().contains("expectedTestException")); + assertTrue(buffer.toString().contains(expectedTestException)); } finally { + + clusterClient.shutdown(); restoreStdOutAndStdErr(); } } @@ -128,13 +131,14 @@ public void testTriggerSavepointFailureIllegalJobID() throws Exception { replaceStdOutAndStdErr(); try { - CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + CliFrontend frontend = new MockedCliFrontend(new StandaloneClusterClient( + new Configuration(), + new TestingHighAvailabilityServices())); String[] parameters = { "invalid job id" }; int returnCode = frontend.savepoint(parameters); - assertTrue(returnCode != 0); - assertTrue(buffer.toString().contains("not a valid ID")); + assertNotEquals(0, returnCode); } finally { restoreStdOutAndStdErr(); @@ -149,23 +153,26 @@ public void testTriggerSavepointFailureIllegalJobID() throws Exception { public void testTriggerSavepointCustomTarget() throws Exception { replaceStdOutAndStdErr(); - try { - JobID jobId = new JobID(); + JobID jobId = new JobID(); - String savepointDirectory = "customTargetDirectory"; + String savepointDirectory = "customTargetDirectory"; - MockedCliFrontend frontend = new SavepointTestCliFrontend(savepointDirectory); + final ClusterClient clusterClient = createClusterClient(savepointDirectory); + + try { + MockedCliFrontend frontend = new MockedCliFrontend(clusterClient); String[] parameters = { jobId.toString(), savepointDirectory }; - int returnCode = frontend.savepoint(parameters); + frontend.savepoint(parameters); - assertEquals(0, returnCode); - verify(frontend.client, times(1)) + verify(clusterClient, times(1)) .triggerSavepoint(eq(jobId), eq(savepointDirectory)); assertTrue(buffer.toString().contains(savepointDirectory)); } finally { + clusterClient.shutdown(); + restoreStdOutAndStdErr(); } } @@ -178,65 +185,30 @@ public void testTriggerSavepointCustomTarget() throws Exception { public void testDisposeSavepointSuccess() throws Exception { replaceStdOutAndStdErr(); - try { - String savepointPath = "expectedSavepointPath"; - ActorGateway jobManager = mock(ActorGateway.class); + String savepointPath = "expectedSavepointPath"; - Promise triggerResponse = new scala.concurrent.impl.Promise.DefaultPromise<>(); + final CompletableFuture disposeCallFuture = new CompletableFuture<>(); - when(jobManager.ask( - Mockito.eq(new DisposeSavepoint(savepointPath)), - any(FiniteDuration.class))).thenReturn(triggerResponse.future()); + ClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> { + disposeCallFuture.complete(Acknowledge.get()); + return CompletableFuture.completedFuture(Acknowledge.get()); + }); - triggerResponse.success(getDisposeSavepointSuccess()); + try { - CliFrontend frontend = new MockCliFrontend( - CliFrontendTestUtils.getConfigDir(), jobManager); + CliFrontend frontend = new MockedCliFrontend(clusterClient); String[] parameters = { "-d", savepointPath }; - int returnCode = frontend.savepoint(parameters); + frontend.savepoint(parameters); - assertEquals(0, returnCode); - verify(jobManager, times(1)).ask( - Mockito.eq(new DisposeSavepoint(savepointPath)), - any(FiniteDuration.class)); + disposeCallFuture.get(); String outMsg = buffer.toString(); assertTrue(outMsg.contains(savepointPath)); assertTrue(outMsg.contains("disposed")); } finally { - restoreStdOutAndStdErr(); - } - } - - /** - * Tests that a disposal failure due a ClassNotFoundException triggers a - * note about the JAR option. - */ - @Test - public void testDisposeClassNotFoundException() throws Exception { - replaceStdOutAndStdErr(); - - try { - Future classNotFoundFailure = Futures - .successful(new DisposeSavepointFailure(new ClassNotFoundException("Test exception"))); - - ActorGateway jobManager = mock(ActorGateway.class); - when(jobManager.ask(any(DisposeSavepoint.class), any(FiniteDuration.class))) - .thenReturn(classNotFoundFailure); - - CliFrontend frontend = new MockCliFrontend(CliFrontendTestUtils.getConfigDir(), jobManager); - - String[] parameters = { "-d", "any-path" }; - - int returnCode = frontend.savepoint(parameters); - assertTrue(returnCode != 0); - - String out = buffer.toString(); - assertTrue(out.contains("Please provide the program jar with which you have created " + - "the savepoint via -j for disposal")); - } finally { + clusterClient.shutdown(); restoreStdOutAndStdErr(); } } @@ -248,23 +220,32 @@ public void testDisposeClassNotFoundException() throws Exception { public void testDisposeWithJar() throws Exception { replaceStdOutAndStdErr(); - try { - ActorGateway jobManager = mock(ActorGateway.class); - when(jobManager.ask(any(DisposeSavepoint.class), any(FiniteDuration.class))) - .thenReturn(Futures.successful(JobManagerMessages.getDisposeSavepointSuccess())); + final CompletableFuture disposeSavepointFuture = new CompletableFuture<>(); - CliFrontend frontend = new MockCliFrontend(CliFrontendTestUtils.getConfigDir(), jobManager); + final ClusterClient clusterClient = new DisposeSavepointClusterClient( + (String savepointPath, Time timeout) -> { + disposeSavepointFuture.complete(savepointPath); + return CompletableFuture.completedFuture(Acknowledge.get()); + }); + + try { + CliFrontend frontend = new MockedCliFrontend(clusterClient); // Fake JAR file File f = tmp.newFile(); ZipOutputStream out = new ZipOutputStream(new FileOutputStream(f)); out.close(); - String[] parameters = { "-d", "any-path", "-j", f.getAbsolutePath() }; + final String disposePath = "any-path"; + String[] parameters = { "-d", disposePath, "-j", f.getAbsolutePath() }; - int returnCode = frontend.savepoint(parameters); - assertEquals(0, returnCode); + frontend.savepoint(parameters); + + final String actualSavepointPath = disposeSavepointFuture.get(); + + assertEquals(disposePath, actualSavepointPath); } finally { + clusterClient.shutdown(); restoreStdOutAndStdErr(); } } @@ -273,92 +254,44 @@ public void testDisposeWithJar() throws Exception { public void testDisposeSavepointFailure() throws Exception { replaceStdOutAndStdErr(); - try { - String savepointPath = "expectedSavepointPath"; - ActorGateway jobManager = mock(ActorGateway.class); - - Promise triggerResponse = new scala.concurrent.impl.Promise.DefaultPromise<>(); - - when(jobManager.ask( - Mockito.eq(new DisposeSavepoint(savepointPath)), - any(FiniteDuration.class))) - .thenReturn(triggerResponse.future()); - - Exception testException = new Exception("expectedTestException"); - - triggerResponse.success(new DisposeSavepointFailure(testException)); + String savepointPath = "expectedSavepointPath"; - CliFrontend frontend = new MockCliFrontend( - CliFrontendTestUtils.getConfigDir(), jobManager); + Exception testException = new Exception("expectedTestException"); - String[] parameters = { "-d", savepointPath }; - int returnCode = frontend.savepoint(parameters); - - assertTrue(returnCode != 0); - verify(jobManager, times(1)).ask( - Mockito.eq(new DisposeSavepoint(savepointPath)), - any(FiniteDuration.class)); - - assertTrue(buffer.toString().contains("expectedTestException")); - } - finally { - restoreStdOutAndStdErr(); - } - } - - @Test - public void testDisposeSavepointFailureUnknownResponse() throws Exception { - replaceStdOutAndStdErr(); + ClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> FutureUtils.completedExceptionally(testException)); try { - String savepointPath = "expectedSavepointPath"; - ActorGateway jobManager = mock(ActorGateway.class); - - Promise triggerResponse = new scala.concurrent.impl.Promise.DefaultPromise<>(); - - when(jobManager.ask( - Mockito.eq(new DisposeSavepoint(savepointPath)), - any(FiniteDuration.class))) - .thenReturn(triggerResponse.future()); - - triggerResponse.success("UNKNOWN RESPONSE"); - - CliFrontend frontend = new MockCliFrontend( - CliFrontendTestUtils.getConfigDir(), jobManager); + CliFrontend frontend = new MockedCliFrontend(clusterClient); String[] parameters = { "-d", savepointPath }; + int returnCode = frontend.savepoint(parameters); - assertTrue(returnCode != 0); - verify(jobManager, times(1)).ask( - Mockito.eq(new DisposeSavepoint(savepointPath)), - any(FiniteDuration.class)); + assertNotEquals(0, returnCode); - String errMsg = buffer.toString(); - assertTrue(errMsg.contains("IllegalStateException")); - assertTrue(errMsg.contains("Unknown JobManager response")); + assertTrue(buffer.toString().contains(testException.getMessage())); } finally { + clusterClient.shutdown(); restoreStdOutAndStdErr(); } - - replaceStdOutAndStdErr(); } // ------------------------------------------------------------------------ - private static class MockCliFrontend extends CliFrontend { + private static final class DisposeSavepointClusterClient extends StandaloneClusterClient { + + final BiFunction> disposeSavepointFunction; - private final ActorGateway mockJobManager; + public DisposeSavepointClusterClient(BiFunction> disposeSavepointFunction) throws Exception { + super(new Configuration(), new TestingHighAvailabilityServices()); - public MockCliFrontend(String configDir, ActorGateway mockJobManager) throws Exception { - super(configDir); - this.mockJobManager = mockJobManager; + this.disposeSavepointFunction = Preconditions.checkNotNull(disposeSavepointFunction); } @Override - protected ActorGateway getJobManagerGateway(CommandLineOptions options) throws Exception { - return mockJobManager; + public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) { + return disposeSavepointFunction.apply(savepointPath, timeout); } } @@ -376,16 +309,21 @@ private static void restoreStdOutAndStdErr() { System.setErr(stdErr); } - private static final class SavepointTestCliFrontend extends MockedCliFrontend { + private static ClusterClient createClusterClient(String expectedResponse) throws Exception { + final ClusterClient clusterClient = mock(ClusterClient.class); - SavepointTestCliFrontend(String expectedResponse) throws Exception { - when(client.triggerSavepoint(any(JobID.class), anyString())) - .thenReturn(CompletableFuture.completedFuture(expectedResponse)); - } + when(clusterClient.triggerSavepoint(any(JobID.class), anyString())) + .thenReturn(CompletableFuture.completedFuture(expectedResponse)); - SavepointTestCliFrontend(Exception expectedException) throws Exception { - when(client.triggerSavepoint(any(JobID.class), anyString())) - .thenReturn(FutureUtils.completedExceptionally(expectedException)); - } + return clusterClient; + } + + private static ClusterClient createFailingClusterClient(Exception expectedException) throws Exception { + final ClusterClient clusterClient = mock(ClusterClient.class); + + when(clusterClient.triggerSavepoint(any(JobID.class), anyString())) + .thenReturn(FutureUtils.completedExceptionally(expectedException)); + + return clusterClient; } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/util/MockedCliFrontend.java b/flink-clients/src/test/java/org/apache/flink/client/util/MockedCliFrontend.java index c121c25fe6f59..663746b6bdc57 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/util/MockedCliFrontend.java +++ b/flink-clients/src/test/java/org/apache/flink/client/util/MockedCliFrontend.java @@ -39,6 +39,11 @@ public class MockedCliFrontend extends CliFrontend { public final ClusterClient client; + public MockedCliFrontend(ClusterClient clusterClient) throws Exception { + super(CliFrontendTestUtils.getConfigDir()); + this.client = clusterClient; + } + protected MockedCliFrontend() throws Exception { super(CliFrontendTestUtils.getConfigDir()); this.client = mock(ClusterClient.class); From 30fca8944370e181aa0ddcc964ff182d0a52803a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 10 Jan 2018 16:14:13 +0100 Subject: [PATCH 07/12] fixup! [FLINK-8332] [flip6] Move savepoint dispose into ClusterClient --- .../src/main/java/org/apache/flink/client/CliFrontend.java | 4 ++-- .../org/apache/flink/client/CliFrontendSavepointTest.java | 7 +++++-- 2 files changed, 7 insertions(+), 4 deletions(-) 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 70dacf3735ba8..f7fcb60f78f78 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 @@ -671,7 +671,7 @@ protected int savepoint(String[] args) { if (cleanedArgs.length >= 1) { String jobIdString = cleanedArgs[0]; try { - jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); + jobId = JobID.fromHexString(jobIdString); } catch (Exception e) { return handleArgException(new IllegalArgumentException( "Error: The value for the Job ID is not a valid ID.")); @@ -739,7 +739,7 @@ private int disposeSavepoint(ClusterClient clusterClient, String savepointPath) logAndSysout("Disposing savepoint '" + savepointPath + "'."); - CompletableFuture disposeFuture = null; + final CompletableFuture disposeFuture; try { disposeFuture = clusterClient.disposeSavepoint(savepointPath, FutureUtils.toTime(clientTimeout)); } catch (Exception e) { diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java index 8a81e1b066dab..5b16e9f2fe2a8 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java @@ -84,8 +84,9 @@ public void testTriggerSavepointSuccess() throws Exception { MockedCliFrontend frontend = new MockedCliFrontend(clusterClient); String[] parameters = { jobId.toString() }; - frontend.savepoint(parameters); + int returnCode = frontend.savepoint(parameters); + assertEquals(0, returnCode); verify(clusterClient, times(1)) .triggerSavepoint(eq(jobId), isNull(String.class)); @@ -138,6 +139,7 @@ public void testTriggerSavepointFailureIllegalJobID() throws Exception { String[] parameters = { "invalid job id" }; int returnCode = frontend.savepoint(parameters); + assertTrue(buffer.toString().contains("not a valid ID")); assertNotEquals(0, returnCode); } finally { @@ -163,8 +165,9 @@ public void testTriggerSavepointCustomTarget() throws Exception { MockedCliFrontend frontend = new MockedCliFrontend(clusterClient); String[] parameters = { jobId.toString(), savepointDirectory }; - frontend.savepoint(parameters); + int returnCode = frontend.savepoint(parameters); + assertEquals(0, returnCode); verify(clusterClient, times(1)) .triggerSavepoint(eq(jobId), eq(savepointDirectory)); From 23282191c68051545ee1e5ce5784691fbb70d902 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 10 Jan 2018 16:17:00 +0100 Subject: [PATCH 08/12] fixup! [FLINK-8332] [flip6] Move savepoint dispose into ClusterClient --- .../apache/flink/client/CliFrontendSavepointTest.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java index 5b16e9f2fe2a8..8a3c8708ea27b 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java @@ -190,12 +190,8 @@ public void testDisposeSavepointSuccess() throws Exception { String savepointPath = "expectedSavepointPath"; - final CompletableFuture disposeCallFuture = new CompletableFuture<>(); - - ClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> { - disposeCallFuture.complete(Acknowledge.get()); - return CompletableFuture.completedFuture(Acknowledge.get()); - }); + ClusterClient clusterClient = new DisposeSavepointClusterClient( + (String path, Time timeout) -> CompletableFuture.completedFuture(Acknowledge.get())); try { @@ -204,8 +200,6 @@ public void testDisposeSavepointSuccess() throws Exception { String[] parameters = { "-d", savepointPath }; frontend.savepoint(parameters); - disposeCallFuture.get(); - String outMsg = buffer.toString(); assertTrue(outMsg.contains(savepointPath)); assertTrue(outMsg.contains("disposed")); From 9b50957ed4e45784c3039d3f9b494f5c365a1510 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 2 Jan 2018 07:42:18 +0100 Subject: [PATCH 09/12] [FLINK-8333] [flip6] Separate deployment options from command options This commit separates the parsing of command options and deployment options into two steps. This makes it easier to make the CustomCommandLines non-static. Moreover, this commit moves the CliFrontend into the cli sub package. --- .../flink/client/cli/CliArgsException.java | 4 + .../flink/client/{ => cli}/CliFrontend.java | 396 ++++++++---------- .../flink/client/cli/CliFrontendParser.java | 47 ++- .../apache/flink/client/cli/DefaultCLI.java | 2 +- .../flink/client/cli/Flip6DefaultCLI.java | 2 +- .../flink/client/CliFrontendRunTest.java | 170 -------- .../CliFrontendAddressConfigurationTest.java | 6 +- .../{ => cli}/CliFrontendCancelTest.java | 119 +++--- .../client/{ => cli}/CliFrontendInfoTest.java | 54 +-- .../client/{ => cli}/CliFrontendListTest.java | 22 +- .../CliFrontendPackageProgramTest.java | 37 +- .../flink/client/cli/CliFrontendRunTest.java | 168 ++++++++ .../{ => cli}/CliFrontendSavepointTest.java | 4 +- .../client/{ => cli}/CliFrontendStopTest.java | 76 ++-- .../{ => cli}/CliFrontendTestUtils.java | 2 +- .../{ => cli}/util/MockedCliFrontend.java | 6 +- .../client/program/PackagedProgramTest.java | 2 +- flink-dist/src/main/flink-bin/bin/flink | 2 +- flink-dist/src/main/flink-bin/bin/flink.bat | 2 +- .../apache/flink/api/scala/FlinkShell.scala | 3 +- ...iFrontendYarnAddressConfigurationTest.java | 6 +- .../flink/yarn/FlinkYarnSessionCliTest.java | 2 +- .../org/apache/flink/yarn/YarnTestBase.java | 9 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 2 +- 24 files changed, 547 insertions(+), 596 deletions(-) rename flink-clients/src/main/java/org/apache/flink/client/{ => cli}/CliFrontend.java (78%) delete mode 100644 flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/CliFrontendAddressConfigurationTest.java (93%) rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/CliFrontendCancelTest.java (53%) rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/CliFrontendInfoTest.java (69%) rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/CliFrontendListTest.java (81%) rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/CliFrontendPackageProgramTest.java (91%) create mode 100644 flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/CliFrontendSavepointTest.java (99%) rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/CliFrontendStopTest.java (66%) rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/CliFrontendTestUtils.java (98%) rename flink-clients/src/test/java/org/apache/flink/client/{ => cli}/util/MockedCliFrontend.java (93%) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliArgsException.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliArgsException.java index 027be07f7a02f..9efd6a312864d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliArgsException.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliArgsException.java @@ -28,4 +28,8 @@ public class CliArgsException extends Exception { public CliArgsException(String message) { super(message); } + + public CliArgsException(String message, Throwable cause) { + super(message, cause); + } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java similarity index 78% rename from flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java rename to flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index f7fcb60f78f78..b0039d8b7295e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; @@ -24,19 +24,6 @@ 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.client.cli.CancelOptions; -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.DefaultCLI; -import org.apache.flink.client.cli.Flip6DefaultCLI; -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.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; @@ -60,6 +47,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.runtime.util.EnvironmentInformation; @@ -69,6 +57,7 @@ import org.apache.flink.util.StringUtils; import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -211,53 +200,45 @@ public String getConfigurationDirectory() { * * @param args Command line arguments for the run action. */ - protected int run(String[] args) { + protected int run(String[] args) throws Exception { LOG.info("Running 'run' command."); - RunOptions options; - try { - options = CliFrontendParser.parseRunCommand(args); - } - catch (CliArgsException e) { - return handleArgException(e); - } - catch (Throwable t) { - return handleError(t); - } + final Options commandOptions = CliFrontendParser.getRunCommandOptions(); + + final CommandLine commandLine = CliFrontendParser.parse(commandOptions, args, true); + + final RunOptions runOptions = new RunOptions(commandLine); // evaluate help flag - if (options.isPrintHelp()) { + if (runOptions.isPrintHelp()) { CliFrontendParser.printHelpForRun(); return 0; } - if (options.getJarFilePath() == null) { - return handleArgException(new CliArgsException("The program JAR file was not specified.")); + if (runOptions.getJarFilePath() == null) { + throw new CliArgsException("The program JAR file was not specified."); } - PackagedProgram program; + final PackagedProgram program; try { LOG.info("Building program from JAR file"); - program = buildProgram(options); + program = buildProgram(runOptions); } catch (FileNotFoundException e) { - return handleArgException(e); - } - catch (Throwable t) { - return handleError(t); + throw new CliArgsException("Could not build the program from JAR file.", e); } - ClusterClient client = null; - try { + final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine); + ClusterClient client = createClient(customCommandLine, commandLine, program); - client = createClient(options, program); - client.setPrintStatusDuringExecution(options.getStdoutLogging()); - client.setDetached(options.getDetachedMode()); + try { + client.setPrintStatusDuringExecution(runOptions.getStdoutLogging()); + client.setDetached(runOptions.getDetachedMode()); LOG.debug("Client slots is set to {}", client.getMaxSlots()); - LOG.debug(options.getSavepointRestoreSettings().toString()); + LOG.debug(runOptions.getSavepointRestoreSettings().toString()); - int userParallelism = options.getParallelism(); + int userParallelism = runOptions.getParallelism(); LOG.debug("User parallelism is set to {}", userParallelism); if (client.getMaxSlots() != -1 && userParallelism == -1) { logAndSysout("Using the parallelism provided by the remote cluster (" @@ -270,19 +251,13 @@ protected int run(String[] args) { return executeProgram(program, client, userParallelism); } - catch (Throwable t) { - return handleError(t); - } finally { - if (client != null) { - try { - client.shutdown(); - } catch (Exception e) { - LOG.warn("Could not properly shut down the cluster client.", e); - } - } - if (program != null) { - program.deleteExtractedLibraries(); + program.deleteExtractedLibraries(); + + try { + client.shutdown(); + } catch (Exception e) { + LOG.info("Could not properly shut down the client.", e); } } } @@ -292,20 +267,14 @@ protected int run(String[] args) { * * @param args Command line arguments for the info action. */ - protected int info(String[] args) { + protected int info(String[] args) throws CliArgsException, FileNotFoundException, ProgramInvocationException { LOG.info("Running 'info' command."); - // Parse command line options - InfoOptions options; - try { - options = CliFrontendParser.parseInfoCommand(args); - } - catch (CliArgsException e) { - return handleArgException(e); - } - catch (Throwable t) { - return handleError(t); - } + final Options commandOptions = CliFrontendParser.getInfoCommandOptions(); + + final CommandLine commandLine = CliFrontendParser.parse(commandOptions, args, true); + + InfoOptions options = new InfoOptions(commandLine); // evaluate help flag if (options.isPrintHelp()) { @@ -319,14 +288,8 @@ protected int info(String[] args) { // -------- build the packaged program ------------- - PackagedProgram program; - try { - LOG.info("Building program from JAR file"); - program = buildProgram(options); - } - catch (Throwable t) { - return handleError(t); - } + LOG.info("Building program from JAR file"); + final PackagedProgram program = buildProgram(options); try { int parallelism = options.getParallelism(); @@ -366,9 +329,6 @@ protected int info(String[] args) { } return 0; } - catch (Throwable t) { - return handleError(t); - } finally { program.deleteExtractedLibraries(); } @@ -379,19 +339,14 @@ protected int info(String[] args) { * * @param args Command line arguments for the list action. */ - protected int list(String[] args) { + protected int list(String[] args) throws Exception { LOG.info("Running 'list' command."); - ListOptions options; - try { - options = CliFrontendParser.parseListCommand(args); - } - catch (CliArgsException e) { - return handleArgException(e); - } - catch (Throwable t) { - return handleError(t); - } + final Options commandOptions = CliFrontendParser.getListCommandOptions(); + + final CommandLine commandLine = CliFrontendParser.parse(commandOptions, args, false); + + ListOptions options = new ListOptions(commandLine); // evaluate help flag if (options.isPrintHelp()) { @@ -408,10 +363,10 @@ protected int list(String[] args) { scheduled = true; } - try { - CustomCommandLine activeCommandLine = getActiveCustomCommandLine(options.getCommandLine()); - ClusterClient client = activeCommandLine.retrieveCluster(options.getCommandLine(), config, configurationDirectory); + final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); + final ClusterClient client = activeCommandLine.retrieveCluster(commandLine, config, configurationDirectory); + try { Collection jobDetails; try { CompletableFuture> jobDetailsFuture = client.listJobs(); @@ -475,9 +430,12 @@ protected int list(String[] args) { } return 0; - } - catch (Throwable t) { - return handleError(t); + } finally { + try { + client.shutdown(); + } catch (Exception e) { + LOG.info("Could not properly shut down the client.", e); + } } } @@ -486,19 +444,14 @@ protected int list(String[] args) { * * @param args Command line arguments for the stop action. */ - protected int stop(String[] args) { + protected int stop(String[] args) throws Exception { LOG.info("Running 'stop' command."); - StopOptions options; - try { - options = CliFrontendParser.parseStopCommand(args); - } - catch (CliArgsException e) { - return handleArgException(e); - } - catch (Throwable t) { - return handleError(t); - } + final Options commandOptions = CliFrontendParser.getStopCommandOptions(); + + final CommandLine commandLine = CliFrontendParser.parse(commandOptions, args, false); + + StopOptions options = new StopOptions(commandLine); // evaluate help flag if (options.isPrintHelp()) { @@ -511,33 +464,28 @@ protected int stop(String[] args) { if (stopArgs.length > 0) { String jobIdString = stopArgs[0]; - try { - jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); - } - catch (Exception e) { - return handleError(e); - } + jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); } else { - return handleArgException(new CliArgsException("Missing JobID")); + throw new CliArgsException("Missing JobID"); } + final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); + + final ClusterClient client = activeCommandLine.retrieveCluster(commandLine, config, configurationDirectory); + try { - CustomCommandLine activeCommandLine = getActiveCustomCommandLine(options.getCommandLine()); - ClusterClient client = activeCommandLine.retrieveCluster(options.getCommandLine(), config, configurationDirectory); - try { - logAndSysout("Stopping job " + jobId + '.'); - client.stop(jobId); - logAndSysout("Stopped job " + jobId + '.'); + logAndSysout("Stopping job " + jobId + '.'); + client.stop(jobId); + logAndSysout("Stopped job " + jobId + '.'); - return 0; - } finally { + return 0; + } finally { + try { client.shutdown(); + } catch (Exception e) { + LOG.info("Could not properly shut down the client.", e); } - - } - catch (Throwable t) { - return handleError(t); } } @@ -546,30 +494,25 @@ protected int stop(String[] args) { * * @param args Command line arguments for the cancel action. */ - protected int cancel(String[] args) { + protected int cancel(String[] args) throws Exception { LOG.info("Running 'cancel' command."); - CancelOptions options; - try { - options = CliFrontendParser.parseCancelCommand(args); - } - catch (CliArgsException e) { - return handleArgException(e); - } - catch (Throwable t) { - return handleError(t); - } + final Options commandOptions = CliFrontendParser.getCancelCommandOptions(); + + final CommandLine commandLine = CliFrontendParser.parse(commandOptions, args, false); + + CancelOptions cancelOptions = new CancelOptions(commandLine); // evaluate help flag - if (options.isPrintHelp()) { + if (cancelOptions.isPrintHelp()) { CliFrontendParser.printHelpForCancel(); return 0; } - String[] cleanedArgs = options.getArgs(); + String[] cleanedArgs = cancelOptions.getArgs(); - boolean withSavepoint = options.isWithSavepoint(); - String targetDirectory = options.getSavepointTargetDirectory(); + boolean withSavepoint = cancelOptions.isWithSavepoint(); + String targetDirectory = cancelOptions.getSavepointTargetDirectory(); JobID jobId; @@ -583,9 +526,7 @@ protected int cancel(String[] args) { try { jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); } catch (Exception e) { - LOG.error("Error: The value for the Job ID is not a valid ID."); - System.out.println("Error: The value for the Job ID is not a valid ID."); - return 1; + throw new CliArgsException("The value for the JobID is not a valid ID: " + e.getMessage()); } } else if (targetDirectory != null) { // Try this for case: cancel -s (default savepoint target dir) @@ -594,42 +535,38 @@ protected int cancel(String[] args) { jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); targetDirectory = null; } catch (Exception e) { - LOG.error("Missing JobID in the command line arguments."); - System.out.println("Error: Specify a Job ID to cancel a job."); - return 1; + throw new CliArgsException("Missing JobID in the command line arguments: " + e.getMessage()); } } else { - LOG.error("Missing JobID in the command line arguments."); - System.out.println("Error: Specify a Job ID to cancel a job."); - return 1; + throw new CliArgsException("Missing JobID in the command line arguments."); } + final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); + final ClusterClient client = activeCommandLine.retrieveCluster(commandLine, config, configurationDirectory); + try { - CustomCommandLine activeCommandLine = getActiveCustomCommandLine(options.getCommandLine()); - ClusterClient client = activeCommandLine.retrieveCluster(options.getCommandLine(), config, configurationDirectory); - try { - if (withSavepoint) { - if (targetDirectory == null) { - logAndSysout("Cancelling job " + jobId + " with savepoint to default savepoint directory."); - } else { - logAndSysout("Cancelling job " + jobId + " with savepoint to " + targetDirectory + '.'); - } - String savepointPath = client.cancelWithSavepoint(jobId, targetDirectory); - logAndSysout("Cancelled job " + jobId + ". Savepoint stored in " + savepointPath + '.'); + if (withSavepoint) { + if (targetDirectory == null) { + logAndSysout("Cancelling job " + jobId + " with savepoint to default savepoint directory."); } else { - logAndSysout("Cancelling job " + jobId + '.'); - client.cancel(jobId); - logAndSysout("Cancelled job " + jobId + '.'); + logAndSysout("Cancelling job " + jobId + " with savepoint to " + targetDirectory + '.'); } + String savepointPath = client.cancelWithSavepoint(jobId, targetDirectory); + logAndSysout("Cancelled job " + jobId + ". Savepoint stored in " + savepointPath + '.'); + } else { + logAndSysout("Cancelling job " + jobId + '.'); + client.cancel(jobId); + logAndSysout("Cancelled job " + jobId + '.'); + } - return 0; - } finally { + return 0; + } finally { + try { client.shutdown(); + } catch (Exception e) { + LOG.info("Could not properly shut down the client.", e); } } - catch (Throwable t) { - return handleError(t); - } } /** @@ -637,17 +574,10 @@ protected int cancel(String[] args) { * * @param args Command line arguments for the cancel action. */ - protected int savepoint(String[] args) { + protected int savepoint(String[] args) throws CliArgsException { LOG.info("Running 'savepoint' command."); - SavepointOptions options; - try { - options = CliFrontendParser.parseSavepointCommand(args); - } catch (CliArgsException e) { - return handleArgException(e); - } catch (Throwable t) { - return handleError(t); - } + SavepointOptions options = CliFrontendParser.parseSavepointCommand(args); // evaluate help flag if (options.isPrintHelp()) { @@ -672,14 +602,12 @@ protected int savepoint(String[] args) { String jobIdString = cleanedArgs[0]; try { jobId = JobID.fromHexString(jobIdString); - } catch (Exception e) { - return handleArgException(new IllegalArgumentException( - "Error: The value for the Job ID is not a valid ID.")); + } catch (Exception ignored) { + throw new CliArgsException("Error: The value for the Job ID is not a valid ID."); } } else { - return handleArgException(new IllegalArgumentException( - "Error: The value for the Job ID is not a valid ID. " + - "Specify a Job ID to trigger a savepoint.")); + throw new CliArgsException("Error: The value for the Job ID is not a valid ID. " + + "Specify a Job ID to trigger a savepoint."); } String savepointDirectory = null; @@ -730,8 +658,7 @@ private int triggerSavepoint(ClusterClient clusterClient, JobID jobId, String sa } /** - * Sends a {@link org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint} - * message to the job manager. + * Sends a {@link JobManagerMessages.DisposeSavepoint} message to the job manager. */ private int disposeSavepoint(ClusterClient clusterClient, String savepointPath) { Preconditions.checkNotNull(savepointPath, "Missing required argument: savepoint path. " + @@ -859,27 +786,26 @@ protected ClusterClient retrieveClient(CommandLineOptions options) { /** * Creates a {@link ClusterClient} object from the given command line options and other parameters. - * @param options Command line options + * @param customCommandLine custom command line to use to retrieve the client + * @param commandLine command line to use * @param program The program for which to create the client. * @throws Exception */ protected ClusterClient createClient( - CommandLineOptions options, + CustomCommandLine customCommandLine, + CommandLine commandLine, PackagedProgram program) throws Exception { - // Get the custom command-line (e.g. Standalone/Yarn/Mesos) - CustomCommandLine activeCommandLine = getActiveCustomCommandLine(options.getCommandLine()); - ClusterClient client; try { - client = activeCommandLine.retrieveCluster(options.getCommandLine(), config, configurationDirectory); + client = customCommandLine.retrieveCluster(commandLine, config, configurationDirectory); logAndSysout("Cluster configuration: " + client.getClusterIdentifier()); } catch (UnsupportedOperationException e) { try { String applicationName = "Flink Application: " + program.getMainClassName(); - client = activeCommandLine.createCluster( + client = customCommandLine.createCluster( applicationName, - options.getCommandLine(), + commandLine, config, configurationDirectory, program.getAllLibraries()); @@ -912,7 +838,7 @@ protected ClusterClient createClient( * @param e The exception to display. * @return The return code for the process. */ - private int handleArgException(Exception e) { + private static int handleArgException(Exception e) { LOG.error("Invalid command line arguments. " + (e.getMessage() == null ? "" : e.getMessage())); System.out.println(e.getMessage()); @@ -927,7 +853,7 @@ private int handleArgException(Exception e) { * @param e The exception to display. * @return The return code for the process. */ - private int handleParametrizationException(ProgramParametrizationException e) { + private static int handleParametrizationException(ProgramParametrizationException e) { System.err.println(e.getMessage()); return 1; } @@ -937,7 +863,7 @@ private int handleParametrizationException(ProgramParametrizationException e) { * * @return The return code for the process. */ - private int handleMissingJobException() { + private static int handleMissingJobException() { System.err.println(); System.err.println("The program didn't contain a Flink job. " + "Perhaps you forgot to call execute() on the execution environment."); @@ -950,7 +876,7 @@ private int handleMissingJobException() { * @param t The exception to display. * @return The return code for the process. */ - private int handleError(Throwable t) { + private static int handleError(Throwable t) { LOG.error("Error while running the command.", t); System.err.println(); @@ -973,7 +899,7 @@ private int handleError(Throwable t) { return 1; } - private void logAndSysout(String message) { + private static void logAndSysout(String message) { LOG.info(message); System.out.println(message); } @@ -1003,40 +929,50 @@ public int parseParameters(String[] args) { // remove action from parameters final String[] params = Arrays.copyOfRange(args, 1, args.length); - // do action - switch (action) { - case ACTION_RUN: - return run(params); - case ACTION_LIST: - return list(params); - case ACTION_INFO: - return info(params); - case ACTION_CANCEL: - return cancel(params); - case ACTION_STOP: - return stop(params); - case ACTION_SAVEPOINT: - return savepoint(params); - case "-h": - case "--help": - CliFrontendParser.printHelp(); - return 0; - case "-v": - case "--version": - String version = EnvironmentInformation.getVersion(); - String commitID = EnvironmentInformation.getRevisionInformation().commitId; - System.out.print("Version: " + version); - System.out.println(!commitID.equals(EnvironmentInformation.UNKNOWN) ? ", Commit ID: " + commitID : ""); - return 0; - default: - System.out.printf("\"%s\" is not a valid action.\n", action); - System.out.println(); - System.out.println("Valid actions are \"run\", \"list\", \"info\", \"savepoint\", \"stop\", or \"cancel\"."); - System.out.println(); - System.out.println("Specify the version option (-v or --version) to print Flink version."); - System.out.println(); - System.out.println("Specify the help option (-h or --help) to get help on the command."); - return 1; + try { + // do action + switch (action) { + case ACTION_RUN: + return run(params); + case ACTION_LIST: + return list(params); + case ACTION_INFO: + return info(params); + case ACTION_CANCEL: + return cancel(params); + case ACTION_STOP: + return stop(params); + case ACTION_SAVEPOINT: + return savepoint(params); + case "-h": + case "--help": + CliFrontendParser.printHelp(); + return 0; + case "-v": + case "--version": + String version = EnvironmentInformation.getVersion(); + String commitID = EnvironmentInformation.getRevisionInformation().commitId; + System.out.print("Version: " + version); + System.out.println(commitID.equals(EnvironmentInformation.UNKNOWN) ? "" : ", Commit ID: " + commitID); + return 0; + default: + System.out.printf("\"%s\" is not a valid action.\n", action); + System.out.println(); + System.out.println("Valid actions are \"run\", \"list\", \"info\", \"savepoint\", \"stop\", or \"cancel\"."); + System.out.println(); + System.out.println("Specify the version option (-v or --version) to print Flink version."); + System.out.println(); + System.out.println("Specify the help option (-h or --help) to get help on the command."); + return 1; + } + } catch (CliArgsException ce) { + return handleArgException(ce); + } catch (ProgramParametrizationException ppe) { + return handleParametrizationException(ppe); + } catch (ProgramMissingJobException pmje) { + return handleMissingJobException(); + } catch (Exception e) { + return handleError(e); } } 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 1aec391586530..da2b064684cd5 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 @@ -18,7 +18,6 @@ package org.apache.flink.client.cli; -import org.apache.flink.client.CliFrontend; import org.apache.flink.configuration.CoreOptions; import org.apache.commons.cli.CommandLine; @@ -137,12 +136,12 @@ public class CliFrontendParser { CANCEL_WITH_SAVEPOINT_OPTION.setOptionalArg(true); } - private static final Options RUN_OPTIONS = getRunOptions(buildGeneralOptions(new Options())); - private static final Options INFO_OPTIONS = getInfoOptions(buildGeneralOptions(new Options())); - private static final Options LIST_OPTIONS = getListOptions(buildGeneralOptions(new Options())); - private static final Options CANCEL_OPTIONS = getCancelOptions(buildGeneralOptions(new Options())); - private static final Options STOP_OPTIONS = getStopOptions(buildGeneralOptions(new Options())); - private static final Options SAVEPOINT_OPTIONS = getSavepointOptions(buildGeneralOptions(new Options())); + private static final Options RUN_OPTIONS = getRunCommandOptions(); + private static final Options INFO_OPTIONS = getInfoCommandOptions(); + private static final Options LIST_OPTIONS = getListCommandOptions(); + private static final Options CANCEL_OPTIONS = getCancelCommandOptions(); + private static final Options STOP_OPTIONS = getStopCommandOptions(); + private static final Options SAVEPOINT_OPTIONS = getSavepointCommandOptions(); private static Options buildGeneralOptions(Options options) { options.addOption(HELP_OPTION); @@ -177,6 +176,10 @@ private static Options getProgramSpecificOptionsWithoutDeprecatedOptions(Options return options; } + static Options getRunCommandOptions() { + return getRunOptions(buildGeneralOptions(new Options())); + } + private static Options getRunOptions(Options options) { options = getProgramSpecificOptions(options); options.addOption(SAVEPOINT_PATH_OPTION); @@ -191,12 +194,20 @@ private static Options getJobManagerAddressOption(Options options) { return options; } + static Options getInfoCommandOptions() { + return getInfoOptions(buildGeneralOptions(new Options())); + } + private static Options getInfoOptions(Options options) { options = getProgramSpecificOptions(options); options = getJobManagerAddressOption(options); return addCustomCliOptions(options, false); } + static Options getListCommandOptions() { + return getListOptions(buildGeneralOptions(new Options())); + } + private static Options getListOptions(Options options) { options.addOption(RUNNING_OPTION); options.addOption(SCHEDULED_OPTION); @@ -204,17 +215,29 @@ private static Options getListOptions(Options options) { return addCustomCliOptions(options, false); } + static Options getCancelCommandOptions() { + return getCancelOptions(buildGeneralOptions(new Options())); + } + private static Options getCancelOptions(Options options) { options.addOption(CANCEL_WITH_SAVEPOINT_OPTION); options = getJobManagerAddressOption(options); return addCustomCliOptions(options, false); } + static Options getStopCommandOptions() { + return getStopOptions(buildGeneralOptions(new Options())); + } + private static Options getStopOptions(Options options) { options = getJobManagerAddressOption(options); return addCustomCliOptions(options, false); } + static Options getSavepointCommandOptions() { + return getSavepointOptions(buildGeneralOptions(new Options())); + } + private static Options getSavepointOptions(Options options) { options = getJobManagerAddressOption(options); options.addOption(SAVEPOINT_DISPOSE_OPTION); @@ -479,4 +502,14 @@ public static InfoOptions parseInfoCommand(String[] args) throws CliArgsExceptio } } + public static CommandLine parse(Options options, String[] args, boolean stopAtNonOptions) throws CliArgsException { + final DefaultParser parser = new DefaultParser(); + + try { + return parser.parse(options, args, stopAtNonOptions); + } catch (ParseException e) { + throw new CliArgsException(e.getMessage()); + } + } + } diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java index a5d8a30e233c3..1d25752e8f8f1 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java @@ -32,7 +32,7 @@ import java.net.URL; import java.util.List; -import static org.apache.flink.client.CliFrontend.setJobManagerAddressInConfig; +import static org.apache.flink.client.cli.CliFrontend.setJobManagerAddressInConfig; /** * The default CLI which is used for interaction with standalone clusters. diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java index 5fb9dfce7ff08..413b465ac37ce 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java @@ -33,7 +33,7 @@ import java.net.URL; import java.util.List; -import static org.apache.flink.client.CliFrontend.setJobManagerAddressInConfig; +import static org.apache.flink.client.cli.CliFrontend.setJobManagerAddressInConfig; /** * The default CLI which is used for interaction with standalone clusters. 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 deleted file mode 100644 index 0edc44483e4ec..0000000000000 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendRunTest.java +++ /dev/null @@ -1,170 +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.client; - -import org.apache.flink.client.cli.CliFrontendParser; -import org.apache.flink.client.cli.Flip6DefaultCLI; -import org.apache.flink.client.cli.RunOptions; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; - -import org.junit.BeforeClass; -import org.junit.Test; - -import static org.apache.flink.client.CliFrontendTestUtils.getTestJarPath; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Tests for the RUN command. - */ -public class CliFrontendRunTest { - - @BeforeClass - public static void init() { - CliFrontendTestUtils.pipeSystemOutToNull(); - } - - @Test - public void testRun() { - try { - // test unrecognized option - { - String[] parameters = {"-v", "-l", "-a", "some", "program", "arguments"}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - int retCode = testFrontend.run(parameters); - assertNotEquals(0, retCode); - } - - // test without parallelism - { - String[] parameters = {"-v", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(1, true, false); - assertEquals(0, testFrontend.run(parameters)); - } - - // test configure parallelism - { - String[] parameters = {"-v", "-p", "42", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(42, true, false); - assertEquals(0, testFrontend.run(parameters)); - } - - // test configure sysout logging - { - String[] parameters = {"-p", "2", "-q", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, false, false); - assertEquals(0, testFrontend.run(parameters)); - } - - // test detached mode - { - String[] parameters = {"-p", "2", "-d", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, true, true); - assertEquals(0, testFrontend.run(parameters)); - } - - // test configure parallelism with non integer value - { - String[] parameters = {"-v", "-p", "text", getTestJarPath()}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - assertNotEquals(0, testFrontend.run(parameters)); - } - - // test configure parallelism with overflow integer value - { - String[] parameters = {"-v", "-p", "475871387138", getTestJarPath()}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - assertNotEquals(0, testFrontend.run(parameters)); - } - - // test configure savepoint path (no ignore flag) - { - String[] parameters = {"-s", "expectedSavepointPath", getTestJarPath()}; - RunOptions options = CliFrontendParser.parseRunCommand(parameters); - SavepointRestoreSettings savepointSettings = options.getSavepointRestoreSettings(); - assertTrue(savepointSettings.restoreSavepoint()); - assertEquals("expectedSavepointPath", savepointSettings.getRestorePath()); - assertFalse(savepointSettings.allowNonRestoredState()); - } - - // test configure savepoint path (with ignore flag) - { - String[] parameters = {"-s", "expectedSavepointPath", "-n", getTestJarPath()}; - RunOptions options = CliFrontendParser.parseRunCommand(parameters); - SavepointRestoreSettings savepointSettings = options.getSavepointRestoreSettings(); - assertTrue(savepointSettings.restoreSavepoint()); - assertEquals("expectedSavepointPath", savepointSettings.getRestorePath()); - assertTrue(savepointSettings.allowNonRestoredState()); - } - - // test jar arguments - { - String[] parameters = - {"-m", "localhost:6123", getTestJarPath(), "-arg1", "value1", "justavalue", "--arg2", "value2"}; - RunOptions options = CliFrontendParser.parseRunCommand(parameters); - assertEquals("-arg1", options.getProgramArgs()[0]); - assertEquals("value1", options.getProgramArgs()[1]); - assertEquals("justavalue", options.getProgramArgs()[2]); - assertEquals("--arg2", options.getProgramArgs()[3]); - assertEquals("value2", options.getProgramArgs()[4]); - } - - // test flip6 switch - { - String[] parameters = - {"-flip6", getTestJarPath()}; - RunOptions options = CliFrontendParser.parseRunCommand(parameters); - assertTrue(options.getCommandLine().hasOption(Flip6DefaultCLI.FLIP_6.getOpt())); - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private static final class RunTestingCliFrontend extends CliFrontend { - - private final int expectedParallelism; - private final boolean sysoutLogging; - private final boolean isDetached; - - public RunTestingCliFrontend(int expectedParallelism, boolean logging, boolean isDetached) throws Exception { - super(CliFrontendTestUtils.getConfigDir()); - this.expectedParallelism = expectedParallelism; - this.sysoutLogging = logging; - this.isDetached = isDetached; - } - - @Override - protected int executeProgram(PackagedProgram program, ClusterClient client, int parallelism) { - assertEquals(isDetached, client.isDetached()); - assertEquals(sysoutLogging, client.getPrintStatusDuringExecution()); - assertEquals(expectedParallelism, parallelism); - return 0; - } - } -} diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendAddressConfigurationTest.java similarity index 93% rename from flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendAddressConfigurationTest.java index 28c3226b55028..a030442a8617d 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendAddressConfigurationTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendAddressConfigurationTest.java @@ -16,10 +16,8 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; -import org.apache.flink.client.cli.CliFrontendParser; -import org.apache.flink.client.cli.RunOptions; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; @@ -31,7 +29,7 @@ import java.net.InetSocketAddress; -import static org.apache.flink.client.CliFrontendTestUtils.checkJobManagerAddress; +import static org.apache.flink.client.cli.CliFrontendTestUtils.checkJobManagerAddress; import static org.junit.Assert.fail; /** diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java similarity index 53% rename from flink-clients/src/test/java/org/apache/flink/client/CliFrontendCancelTest.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java index f2508dccaca1e..10dba1dea9c3e 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java @@ -16,20 +16,17 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; import org.apache.flink.api.common.JobID; -import org.apache.flink.client.cli.CancelOptions; -import org.apache.flink.client.cli.CliFrontendParser; -import org.apache.flink.client.cli.Flip6DefaultCLI; -import org.apache.flink.client.util.MockedCliFrontend; +import org.apache.flink.client.cli.util.MockedCliFrontend; +import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -42,7 +39,7 @@ /** * Tests for the CANCEL command. */ -public class CliFrontendCancelTest { +public class CliFrontendCancelTest extends TestLogger { @BeforeClass public static void init() { @@ -50,62 +47,46 @@ public static void init() { } @Test - public void testCancel() { - try { - // test unrecognized option - { - String[] parameters = {"-v", "-l"}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - int retCode = testFrontend.cancel(parameters); - assertTrue(retCode != 0); - } - - // test missing job id - { - String[] parameters = {}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - int retCode = testFrontend.cancel(parameters); - assertTrue(retCode != 0); - } + public void testCancel() throws Exception { - // test cancel properly - { - JobID jid = new JobID(); + // test cancel properly + { + JobID jid = new JobID(); - String[] parameters = { jid.toString() }; - CancelTestCliFrontend testFrontend = new CancelTestCliFrontend(false); + String[] parameters = { jid.toString() }; + CancelTestCliFrontend testFrontend = new CancelTestCliFrontend(false); - int retCode = testFrontend.cancel(parameters); - assertTrue(retCode == 0); + int retCode = testFrontend.cancel(parameters); + assertTrue(retCode == 0); - Mockito.verify(testFrontend.client, times(1)).cancel(any(JobID.class)); - } + Mockito.verify(testFrontend.client, times(1)).cancel(any(JobID.class)); + } - // test cancel properly - { - JobID jid = new JobID(); + // test flip6 switch + { + String[] parameters = + {"-flip6", String.valueOf(new JobID())}; + CancelOptions options = CliFrontendParser.parseCancelCommand(parameters); + assertTrue(options.getCommandLine().hasOption(Flip6DefaultCLI.FLIP_6.getOpt())); + } + } - String[] parameters = { jid.toString() }; - CancelTestCliFrontend testFrontend = new CancelTestCliFrontend(true); + @Test(expected = CliArgsException.class) + public void testMissingJobId() throws Exception { + String[] parameters = {}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.cancel(parameters); - int retCode = testFrontend.cancel(parameters); - assertTrue(retCode != 0); + fail("Should have failed."); + } - Mockito.verify(testFrontend.client, times(1)).cancel(any(JobID.class)); - } + @Test(expected = CliArgsException.class) + public void testUnrecognizedOption() throws Exception { + String[] parameters = {"-v", "-l"}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.cancel(parameters); - // test flip6 switch - { - String[] parameters = - {"-flip6", String.valueOf(new JobID())}; - CancelOptions options = CliFrontendParser.parseCancelCommand(parameters); - assertTrue(options.getCommandLine().hasOption(Flip6DefaultCLI.FLIP_6.getOpt())); - } - } - catch (Exception e) { - e.printStackTrace(); - fail("Program caused an exception: " + e.getMessage()); - } + fail("Should have failed with CliArgsException."); } /** @@ -136,20 +117,26 @@ public void testCancelWithSavepoint() throws Exception { Mockito.verify(testFrontend.client, times(1)) .cancelWithSavepoint(any(JobID.class), notNull(String.class)); } + } - { - // Cancel with savepoint (with target directory), but no job ID - String[] parameters = { "-s", "targetDirectory" }; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - assertNotEquals(0, testFrontend.cancel(parameters)); - } + @Test(expected = CliArgsException.class) + public void testCancelWithSavepointWithoutJobId() throws Exception { + // Cancel with savepoint (with target directory), but no job ID + String[] parameters = { "-s", "targetDirectory" }; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.cancel(parameters); - { - // Cancel with savepoint (no target directory) and no job ID - String[] parameters = { "-s" }; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - assertNotEquals(0, testFrontend.cancel(parameters)); - } + fail("Should have failed."); + } + + @Test(expected = CliArgsException.class) + public void testCancelWithSavepointWithoutParameters() throws Exception { + // Cancel with savepoint (no target directory) and no job ID + String[] parameters = { "-s" }; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.cancel(parameters); + + fail("Should have failed."); } private static final class CancelTestCliFrontend extends MockedCliFrontend { diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java similarity index 69% rename from flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java index 5a79bb69d6231..cfca33e7324e3 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendInfoTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java @@ -16,7 +16,9 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; + +import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -29,52 +31,42 @@ /** * Tests for the "info" command. */ -public class CliFrontendInfoTest { +public class CliFrontendInfoTest extends TestLogger { private static PrintStream stdOut; private static PrintStream capture; private static ByteArrayOutputStream buffer; - @Test - public void testErrorCases() { - try { - // test unrecognized option - { - String[] parameters = {"-v", "-l"}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - int retCode = testFrontend.cancel(parameters); - assertTrue(retCode != 0); - } - - // test missing options - { - String[] parameters = {}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - int retCode = testFrontend.cancel(parameters); - assertTrue(retCode != 0); - } - } - catch (Exception e) { - e.printStackTrace(); - fail("Program caused an exception: " + e.getMessage()); - } + @Test(expected = CliArgsException.class) + public void testMissingOption() throws Exception { + String[] parameters = {}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.cancel(parameters); + + fail("Should have failed with CliArgsException"); + } + + @Test(expected = CliArgsException.class) + public void testUnrecognizedOption() throws Exception { + String[] parameters = {"-v", "-l"}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.cancel(parameters); + + fail("Should have failed with CliArgsException"); } @Test - public void testShowExecutionPlan() { + public void testShowExecutionPlan() throws Exception { replaceStdOut(); try { - String[] parameters = new String[] { CliFrontendTestUtils.getTestJarPath(), "-f", "true"}; + String[] parameters = new String[]{CliFrontendTestUtils.getTestJarPath(), "-f", "true"}; CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); int retCode = testFrontend.info(parameters); assertTrue(retCode == 0); assertTrue(buffer.toString().contains("\"parallelism\": \"1\"")); } - catch (Exception e) { - e.printStackTrace(); - fail("Program caused an exception: " + e.getMessage()); - } finally { + finally { restoreStdOut(); } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java similarity index 81% rename from flink-clients/src/test/java/org/apache/flink/client/CliFrontendListTest.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java index b559af1bff34c..3185d233a456d 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; -import org.apache.flink.client.util.MockedCliFrontend; +import org.apache.flink.client.cli.util.MockedCliFrontend; import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; @@ -29,6 +29,7 @@ import java.util.concurrent.CompletableFuture; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Mockito.times; import static org.powermock.api.mockito.PowerMockito.when; @@ -44,14 +45,6 @@ public static void init() { @Test public void testList() throws Exception { - // test unrecognized option - { - String[] parameters = {"-v", "-k"}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - int retCode = testFrontend.list(parameters); - assertTrue(retCode != 0); - } - // test list properly { String[] parameters = {"-r", "-s"}; @@ -63,6 +56,15 @@ public void testList() throws Exception { } } + @Test(expected = CliArgsException.class) + public void testUnrecognizedOption() throws Exception { + String[] parameters = {"-v", "-k"}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.list(parameters); + + fail("Should have failed with an CliArgsException."); + } + private static final class ListTestCliFrontend extends MockedCliFrontend { ListTestCliFrontend() throws Exception { diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java similarity index 91% rename from flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java index a75f49b892810..b27eab2c2db06 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendPackageProgramTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java @@ -16,11 +16,8 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; -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.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; @@ -37,11 +34,11 @@ import java.io.FileNotFoundException; import java.net.URL; -import static org.apache.flink.client.CliFrontendTestUtils.TEST_JAR_CLASSLOADERTEST_CLASS; -import static org.apache.flink.client.CliFrontendTestUtils.TEST_JAR_MAIN_CLASS; -import static org.apache.flink.client.CliFrontendTestUtils.getNonJarFilePath; -import static org.apache.flink.client.CliFrontendTestUtils.getTestJarPath; -import static org.apache.flink.client.CliFrontendTestUtils.pipeSystemOutToNull; +import static org.apache.flink.client.cli.CliFrontendTestUtils.TEST_JAR_CLASSLOADERTEST_CLASS; +import static org.apache.flink.client.cli.CliFrontendTestUtils.TEST_JAR_MAIN_CLASS; +import static org.apache.flink.client.cli.CliFrontendTestUtils.getNonJarFilePath; +import static org.apache.flink.client.cli.CliFrontendTestUtils.getTestJarPath; +import static org.apache.flink.client.cli.CliFrontendTestUtils.pipeSystemOutToNull; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -190,16 +187,12 @@ public void testValidVariantWithNoJarAndNoArgumentsOption() { } } - @Test - public void testNoJarNoArgumentsAtAll() { - try { - CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - assertTrue(frontend.run(new String[0]) != 0); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + @Test(expected = CliArgsException.class) + public void testNoJarNoArgumentsAtAll() throws Exception { + CliFrontend frontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + assertTrue(frontend.run(new String[0]) != 0); + + fail("Should have failed."); } @Test @@ -267,9 +260,9 @@ public void testNonExistingFileWithoutArguments() { * at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:301) * at org.apache.flink.client.program.Client.getOptimizedPlan(Client.java:140) * at org.apache.flink.client.program.Client.getOptimizedPlanAsJson(Client.java:125) - * at org.apache.flink.client.CliFrontend.info(CliFrontend.java:439) - * at org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:931) - * at org.apache.flink.client.CliFrontend.main(CliFrontend.java:951) + * at org.apache.flink.client.cli.CliFrontend.info(CliFrontend.java:439) + * at org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:931) + * at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:951) * Caused by: java.io.IOException: java.lang.RuntimeException: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.io.RCFileInputFormat * at org.apache.hcatalog.mapreduce.HCatInputFormat.setInput(HCatInputFormat.java:102) * at org.apache.hcatalog.mapreduce.HCatInputFormat.setInput(HCatInputFormat.java:54) diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java new file mode 100644 index 0000000000000..d17d999fec44e --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java @@ -0,0 +1,168 @@ +/* + * 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.flink.client.program.ClusterClient; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; + +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.apache.flink.client.cli.CliFrontendTestUtils.getTestJarPath; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests for the RUN command. + */ +public class CliFrontendRunTest { + + @BeforeClass + public static void init() { + CliFrontendTestUtils.pipeSystemOutToNull(); + } + + @Test + public void testRun() throws Exception { + // test without parallelism + { + String[] parameters = {"-v", getTestJarPath()}; + RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(1, true, false); + assertEquals(0, testFrontend.run(parameters)); + } + + // test configure parallelism + { + String[] parameters = {"-v", "-p", "42", getTestJarPath()}; + RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(42, true, false); + assertEquals(0, testFrontend.run(parameters)); + } + + // test configure sysout logging + { + String[] parameters = {"-p", "2", "-q", getTestJarPath()}; + RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, false, false); + assertEquals(0, testFrontend.run(parameters)); + } + + // test detached mode + { + String[] parameters = {"-p", "2", "-d", getTestJarPath()}; + RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(2, true, true); + assertEquals(0, testFrontend.run(parameters)); + } + + // test configure savepoint path (no ignore flag) + { + String[] parameters = {"-s", "expectedSavepointPath", getTestJarPath()}; + RunOptions options = CliFrontendParser.parseRunCommand(parameters); + SavepointRestoreSettings savepointSettings = options.getSavepointRestoreSettings(); + assertTrue(savepointSettings.restoreSavepoint()); + assertEquals("expectedSavepointPath", savepointSettings.getRestorePath()); + assertFalse(savepointSettings.allowNonRestoredState()); + } + + // test configure savepoint path (with ignore flag) + { + String[] parameters = {"-s", "expectedSavepointPath", "-n", getTestJarPath()}; + RunOptions options = CliFrontendParser.parseRunCommand(parameters); + SavepointRestoreSettings savepointSettings = options.getSavepointRestoreSettings(); + assertTrue(savepointSettings.restoreSavepoint()); + assertEquals("expectedSavepointPath", savepointSettings.getRestorePath()); + assertTrue(savepointSettings.allowNonRestoredState()); + } + + // test jar arguments + { + String[] parameters = + {"-m", "localhost:6123", getTestJarPath(), "-arg1", "value1", "justavalue", "--arg2", "value2"}; + RunOptions options = CliFrontendParser.parseRunCommand(parameters); + assertEquals("-arg1", options.getProgramArgs()[0]); + assertEquals("value1", options.getProgramArgs()[1]); + assertEquals("justavalue", options.getProgramArgs()[2]); + assertEquals("--arg2", options.getProgramArgs()[3]); + assertEquals("value2", options.getProgramArgs()[4]); + } + + // test flip6 switch + { + String[] parameters = + {"-flip6", getTestJarPath()}; + RunOptions options = CliFrontendParser.parseRunCommand(parameters); + assertTrue(options.getCommandLine().hasOption(Flip6DefaultCLI.FLIP_6.getOpt())); + } + } + + @Test(expected = CliArgsException.class) + public void testUnrecognizedOption() throws Exception { + // test unrecognized option + String[] parameters = {"-v", "-l", "-a", "some", "program", "arguments"}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.run(parameters); + + fail("Should have failed."); + } + + @Test(expected = CliArgsException.class) + public void testInvalidParallelismOption() throws Exception { + // test configure parallelism with non integer value + String[] parameters = {"-v", "-p", "text", getTestJarPath()}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.run(parameters); + + fail("Should have failed."); + } + + @Test(expected = CliArgsException.class) + public void testParallelismWithOverflow() throws Exception { + // test configure parallelism with overflow integer value + String[] parameters = {"-v", "-p", "475871387138", getTestJarPath()}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.run(parameters); + + fail("Should have failed."); + } + + // -------------------------------------------------------------------------------------------- + + private static final class RunTestingCliFrontend extends CliFrontend { + + private final int expectedParallelism; + private final boolean sysoutLogging; + private final boolean isDetached; + + public RunTestingCliFrontend(int expectedParallelism, boolean logging, boolean isDetached) throws Exception { + super(CliFrontendTestUtils.getConfigDir()); + this.expectedParallelism = expectedParallelism; + this.sysoutLogging = logging; + this.isDetached = isDetached; + } + + @Override + protected int executeProgram(PackagedProgram program, ClusterClient client, int parallelism) { + assertEquals(isDetached, client.isDetached()); + assertEquals(sysoutLogging, client.getPrintStatusDuringExecution()); + assertEquals(expectedParallelism, parallelism); + return 0; + } + } +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java similarity index 99% rename from flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java index 8a3c8708ea27b..bbbb059239d1f 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java @@ -16,13 +16,13 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.cli.util.MockedCliFrontend; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.StandaloneClusterClient; -import org.apache.flink.client.util.MockedCliFrontend; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java similarity index 66% rename from flink-clients/src/test/java/org/apache/flink/client/CliFrontendStopTest.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java index d10b31ca08484..2a90155efa0d9 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendStopTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java @@ -16,22 +16,20 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; import org.apache.flink.api.common.JobID; -import org.apache.flink.client.cli.CliFrontendParser; -import org.apache.flink.client.cli.Flip6DefaultCLI; -import org.apache.flink.client.cli.StopOptions; -import org.apache.flink.client.util.MockedCliFrontend; +import org.apache.flink.client.cli.util.MockedCliFrontend; import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; -import static org.apache.flink.client.CliFrontendTestUtils.pipeSystemOutToNull; +import static org.apache.flink.client.cli.CliFrontendTestUtils.pipeSystemOutToNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.times; import static org.powermock.api.mockito.PowerMockito.doThrow; @@ -48,22 +46,6 @@ public static void setup() { @Test public void testStop() throws Exception { - // test unrecognized option - { - String[] parameters = { "-v", "-l" }; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - int retCode = testFrontend.stop(parameters); - assertTrue(retCode != 0); - } - - // test missing job id - { - String[] parameters = {}; - CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); - int retCode = testFrontend.stop(parameters); - assertTrue(retCode != 0); - } - // test stop properly { JobID jid = new JobID(); @@ -78,18 +60,6 @@ public void testStop() throws Exception { Mockito.verify(testFrontend.client, times(1)).stop(any(JobID.class)); } - // test unknown job Id - { - JobID jid = new JobID(); - - String[] parameters = { jid.toString() }; - StopTestCliFrontend testFrontend = new StopTestCliFrontend(true); - - assertTrue(testFrontend.stop(parameters) != 0); - - Mockito.verify(testFrontend.client, times(1)).stop(any(JobID.class)); - } - // test flip6 switch { String[] parameters = @@ -99,6 +69,44 @@ public void testStop() throws Exception { } } + @Test(expected = CliArgsException.class) + public void testUnrecognizedOption() throws Exception { + // test unrecognized option + String[] parameters = { "-v", "-l" }; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.stop(parameters); + + fail("Should have failed."); + } + + @Test(expected = CliArgsException.class) + public void testMissingJobId() throws Exception { + // test missing job id + String[] parameters = {}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + testFrontend.stop(parameters); + + fail("Should have failed."); + } + + @Test + public void testUnknownJobId() throws Exception { + // test unknown job Id + JobID jid = new JobID(); + + String[] parameters = { jid.toString() }; + StopTestCliFrontend testFrontend = new StopTestCliFrontend(true); + + try { + testFrontend.stop(parameters); + fail("Should have failed."); + } catch (IllegalArgumentException ignored) { + // expected + } + + Mockito.verify(testFrontend.client, times(1)).stop(any(JobID.class)); + } + private static final class StopTestCliFrontend extends MockedCliFrontend { StopTestCliFrontend(boolean reject) throws Exception { diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestUtils.java similarity index 98% rename from flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestUtils.java index 8df39e01ad986..16737dd125ffd 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendTestUtils.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.client; +package org.apache.flink.client.cli; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; diff --git a/flink-clients/src/test/java/org/apache/flink/client/util/MockedCliFrontend.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java similarity index 93% rename from flink-clients/src/test/java/org/apache/flink/client/util/MockedCliFrontend.java rename to flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java index 663746b6bdc57..6d2bcca877095 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/util/MockedCliFrontend.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java @@ -16,10 +16,10 @@ * limitations under the License. */ -package org.apache.flink.client.util; +package org.apache.flink.client.cli.util; -import org.apache.flink.client.CliFrontend; -import org.apache.flink.client.CliFrontendTestUtils; +import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.client.cli.CliFrontendTestUtils; import org.apache.flink.client.cli.CustomCommandLine; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java index e68d1dcbe8d1e..e137adae3d327 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java @@ -18,7 +18,7 @@ package org.apache.flink.client.program; -import org.apache.flink.client.CliFrontendTestUtils; +import org.apache.flink.client.cli.CliFrontendTestUtils; import org.junit.Assert; import org.junit.Test; diff --git a/flink-dist/src/main/flink-bin/bin/flink b/flink-dist/src/main/flink-bin/bin/flink index c9d03c8798d4b..d38217deb9fad 100644 --- a/flink-dist/src/main/flink-bin/bin/flink +++ b/flink-dist/src/main/flink-bin/bin/flink @@ -52,4 +52,4 @@ export FLINK_ROOT_DIR export FLINK_CONF_DIR # Add HADOOP_CLASSPATH to allow the usage of Hadoop file systems -exec $JAVA_RUN $JVM_ARGS "${log_setting[@]}" -classpath "`manglePathList "$CC_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" org.apache.flink.client.CliFrontend "$@" +exec $JAVA_RUN $JVM_ARGS "${log_setting[@]}" -classpath "`manglePathList "$CC_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" org.apache.flink.client.cli.CliFrontend "$@" diff --git a/flink-dist/src/main/flink-bin/bin/flink.bat b/flink-dist/src/main/flink-bin/bin/flink.bat index 279acd2d90502..845d791afec08 100644 --- a/flink-dist/src/main/flink-bin/bin/flink.bat +++ b/flink-dist/src/main/flink-bin/bin/flink.bat @@ -27,6 +27,6 @@ SET JVM_ARGS=-Xmx512m SET FLINK_JM_CLASSPATH=%FLINK_LIB_DIR%\* -java %JVM_ARGS% -cp "%FLINK_JM_CLASSPATH%"; org.apache.flink.client.CliFrontend %* +java %JVM_ARGS% -cp "%FLINK_JM_CLASSPATH%"; org.apache.flink.client.cli.CliFrontend %* endlocal 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 60eaccccc5ba7..43149d96123fd 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 @@ -22,9 +22,8 @@ import java.io._ import java.util.Collections import org.apache.commons.cli.CommandLine -import org.apache.flink.client.cli.CliFrontendParser +import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser} import org.apache.flink.client.program.ClusterClient -import org.apache.flink.client.CliFrontend import org.apache.flink.runtime.minicluster.StandaloneMiniCluster import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration, JobManagerOptions} import org.apache.flink.runtime.minicluster.{FlinkMiniCluster, LocalFlinkMiniCluster} 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 index 56087a18404a5..a255453ed63cb 100644 --- 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 @@ -18,7 +18,7 @@ package org.apache.flink.yarn; -import org.apache.flink.client.CliFrontend; +import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.CommandLineOptions; import org.apache.flink.client.cli.CustomCommandLine; @@ -317,8 +317,8 @@ private static class TestCLI extends CliFrontend { @Override // make method public - public ClusterClient createClient(CommandLineOptions options, PackagedProgram program) throws Exception { - return super.createClient(options, program); + public ClusterClient createClient(CustomCommandLine customCommandLine, CommandLine commandLine, PackagedProgram program) throws Exception { + return super.createClient(customCommandLine, commandLine, program); } @Override 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 3fe8d2f75594a..8541c401eafb7 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 @@ -18,7 +18,7 @@ package org.apache.flink.yarn; -import org.apache.flink.client.CliFrontend; +import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.RunOptions; import org.apache.flink.client.deployment.ClusterSpecification; 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 e0299aae1fb92..d9872a00bb6be 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 @@ -18,8 +18,8 @@ package org.apache.flink.yarn; -import org.apache.flink.client.CliFrontend; -import org.apache.flink.client.cli.CommandLineOptions; +import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.client.cli.CustomCommandLine; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.configuration.ConfigConstants; @@ -31,6 +31,7 @@ import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import akka.actor.Identify; +import org.apache.commons.cli.CommandLine; import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -810,9 +811,9 @@ private static class TestingCLI extends CliFrontend { public TestingCLI() throws Exception {} @Override - protected ClusterClient createClient(CommandLineOptions options, PackagedProgram program) throws Exception { + protected ClusterClient createClient(CustomCommandLine customCommandLine, CommandLine commandLine, PackagedProgram program) throws Exception { // mock the returned ClusterClient to disable shutdown and verify shutdown behavior later on - originalClusterClient = super.createClient(options, program); + originalClusterClient = super.createClient(customCommandLine, commandLine, program); spiedClusterClient = Mockito.spy(originalClusterClient); Mockito.doNothing().when(spiedClusterClient).shutdown(); return spiedClusterClient; diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index c045082d73f65..ef549099b5f0c 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -18,7 +18,7 @@ package org.apache.flink.yarn.cli; -import org.apache.flink.client.CliFrontend; +import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.CustomCommandLine; import org.apache.flink.client.deployment.ClusterSpecification; From 27e367c0f20272fcbe4681403a1bf919db0fa573 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 10 Jan 2018 17:10:24 +0100 Subject: [PATCH 10/12] fixup! [FLINK-8333] [flip6] Separate deployment options from command options --- .../apache/flink/client/cli/CliFrontend.java | 60 ++++++++++--------- .../flink/client/cli/CliFrontendParser.java | 22 ------- .../client/cli/CliFrontendCancelTest.java | 2 +- .../flink/client/cli/CliFrontendStopTest.java | 21 ++++--- .../org/apache/flink/api/common/JobID.java | 15 ++++- 5 files changed, 60 insertions(+), 60 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index b0039d8b7295e..4b21e1039449f 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -54,7 +54,6 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; -import org.apache.flink.util.StringUtils; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; @@ -228,10 +227,12 @@ protected int run(String[] args) throws Exception { throw new CliArgsException("Could not build the program from JAR file.", e); } - final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine); - ClusterClient client = createClient(customCommandLine, commandLine, program); + ClusterClient client = null; try { + final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine); + client = createClient(customCommandLine, commandLine, program); + client.setPrintStatusDuringExecution(runOptions.getStdoutLogging()); client.setDetached(runOptions.getDetachedMode()); LOG.debug("Client slots is set to {}", client.getMaxSlots()); @@ -254,10 +255,12 @@ protected int run(String[] args) throws Exception { finally { program.deleteExtractedLibraries(); - try { - client.shutdown(); - } catch (Exception e) { - LOG.info("Could not properly shut down the client.", e); + if (client != null) { + try { + client.shutdown(); + } catch (Exception e) { + LOG.info("Could not properly shut down the client.", e); + } } } } @@ -464,7 +467,7 @@ protected int stop(String[] args) throws Exception { if (stopArgs.length > 0) { String jobIdString = stopArgs[0]; - jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); + jobId = parseJobId(jobIdString); } else { throw new CliArgsException("Missing JobID"); @@ -523,20 +526,14 @@ protected int cancel(String[] args) throws Exception { // - cancel -s => custom target dir (parsed correctly) if (cleanedArgs.length > 0) { String jobIdString = cleanedArgs[0]; - try { - jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); - } catch (Exception e) { - throw new CliArgsException("The value for the JobID is not a valid ID: " + e.getMessage()); - } + + jobId = parseJobId(jobIdString); } else if (targetDirectory != null) { // Try this for case: cancel -s (default savepoint target dir) String jobIdString = targetDirectory; - try { - jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); - targetDirectory = null; - } catch (Exception e) { - throw new CliArgsException("Missing JobID in the command line arguments: " + e.getMessage()); - } + targetDirectory = null; + + jobId = parseJobId(jobIdString); } else { throw new CliArgsException("Missing JobID in the command line arguments."); } @@ -574,7 +571,7 @@ protected int cancel(String[] args) throws Exception { * * @param args Command line arguments for the cancel action. */ - protected int savepoint(String[] args) throws CliArgsException { + protected int savepoint(String[] args) throws Exception { LOG.info("Running 'savepoint' command."); SavepointOptions options = CliFrontendParser.parseSavepointCommand(args); @@ -600,11 +597,8 @@ protected int savepoint(String[] args) throws CliArgsException { if (cleanedArgs.length >= 1) { String jobIdString = cleanedArgs[0]; - try { - jobId = JobID.fromHexString(jobIdString); - } catch (Exception ignored) { - throw new CliArgsException("Error: The value for the Job ID is not a valid ID."); - } + + jobId = parseJobId(jobIdString); } else { throw new CliArgsException("Error: The value for the Job ID is not a valid ID. " + "Specify a Job ID to trigger a savepoint."); @@ -622,8 +616,6 @@ protected int savepoint(String[] args) throws CliArgsException { return triggerSavepoint(clusterClient, jobId, savepointDirectory); } - } catch (Exception e) { - return handleError(e); } finally { try { clusterClient.shutdown(); @@ -904,6 +896,20 @@ private static void logAndSysout(String message) { System.out.println(message); } + // -------------------------------------------------------------------------------------------- + // Internal methods + // -------------------------------------------------------------------------------------------- + + private JobID parseJobId(String jobIdString) throws CliArgsException { + JobID jobId; + try { + jobId = JobID.fromHexString(jobIdString); + } catch (IllegalArgumentException e) { + throw new CliArgsException(e.getMessage()); + } + return jobId; + } + // -------------------------------------------------------------------------------------------- // Entry point for executable // -------------------------------------------------------------------------------------------- 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 da2b064684cd5..64cfdceb5c6b3 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 @@ -448,17 +448,6 @@ public static RunOptions parseRunCommand(String[] args) throws CliArgsException } } - public static ListOptions parseListCommand(String[] args) throws CliArgsException { - try { - DefaultParser parser = new DefaultParser(); - CommandLine line = parser.parse(LIST_OPTIONS, args, false); - return new ListOptions(line); - } - catch (ParseException e) { - throw new CliArgsException(e.getMessage()); - } - } - public static CancelOptions parseCancelCommand(String[] args) throws CliArgsException { try { DefaultParser parser = new DefaultParser(); @@ -491,17 +480,6 @@ public static SavepointOptions parseSavepointCommand(String[] args) throws CliAr } } - public static InfoOptions parseInfoCommand(String[] args) throws CliArgsException { - try { - DefaultParser parser = new DefaultParser(); - CommandLine line = parser.parse(INFO_OPTIONS, args, true); - return new InfoOptions(line); - } - catch (ParseException e) { - throw new CliArgsException(e.getMessage()); - } - } - public static CommandLine parse(Options options, String[] args, boolean stopAtNonOptions) throws CliArgsException { final DefaultParser parser = new DefaultParser(); diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java index 10dba1dea9c3e..e7d14623a9445 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java @@ -57,7 +57,7 @@ public void testCancel() throws Exception { CancelTestCliFrontend testFrontend = new CancelTestCliFrontend(false); int retCode = testFrontend.cancel(parameters); - assertTrue(retCode == 0); + assertEquals(0,retCode); Mockito.verify(testFrontend.client, times(1)).cancel(any(JobID.class)); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java index 2a90155efa0d9..08de0633235c5 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.client.cli.util.MockedCliFrontend; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; @@ -89,7 +90,7 @@ public void testMissingJobId() throws Exception { fail("Should have failed."); } - @Test + @Test(expected = TestException.class) public void testUnknownJobId() throws Exception { // test unknown job Id JobID jid = new JobID(); @@ -97,21 +98,23 @@ public void testUnknownJobId() throws Exception { String[] parameters = { jid.toString() }; StopTestCliFrontend testFrontend = new StopTestCliFrontend(true); - try { - testFrontend.stop(parameters); - fail("Should have failed."); - } catch (IllegalArgumentException ignored) { - // expected - } + testFrontend.stop(parameters); + fail("Should have failed."); + } + + private static final class TestException extends FlinkException { + private static final long serialVersionUID = -2650760898729937583L; - Mockito.verify(testFrontend.client, times(1)).stop(any(JobID.class)); + TestException(String message) { + super(message); + } } private static final class StopTestCliFrontend extends MockedCliFrontend { StopTestCliFrontend(boolean reject) throws Exception { if (reject) { - doThrow(new IllegalArgumentException("Test exception")).when(client).stop(any(JobID.class)); + doThrow(new TestException("Test Exception")).when(client).stop(any(JobID.class)); } } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobID.java b/flink-core/src/main/java/org/apache/flink/api/common/JobID.java index 78cddfe2af687..36df17a7e157e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobID.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobID.java @@ -20,7 +20,9 @@ import org.apache.flink.annotation.Public; import org.apache.flink.util.AbstractID; + import javax.xml.bind.DatatypeConverter; + import java.nio.ByteBuffer; /** @@ -96,7 +98,18 @@ public static JobID fromByteBuffer(ByteBuffer buf) { return new JobID(lower, upper); } + /** + * Parses a JobID from the given string. + * + * @param hexString string representation of a JobID + * @return Parsed JobID + * @throws IllegalArgumentException if the JobID could not be parsed from the given string + */ public static JobID fromHexString(String hexString) { - return new JobID(DatatypeConverter.parseHexBinary(hexString)); + try { + return new JobID(DatatypeConverter.parseHexBinary(hexString)); + } catch (Exception e) { + throw new IllegalArgumentException("Cannot parse JobID from \"" + hexString + "\".", e); + } } } From 1806e04b9837aef311fcd36fa11a2571af913af1 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 11 Jan 2018 13:21:11 +0100 Subject: [PATCH 11/12] fixup! [FLINK-8333] [flip6] Separate deployment options from command options --- .../client/cli/CliFrontendSavepointTest.java | 25 ++++++++++++------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java index bbbb059239d1f..ff36b7bdcd17e 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java @@ -27,9 +27,11 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; +import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -44,7 +46,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; @@ -114,14 +118,14 @@ public void testTriggerSavepointFailure() throws Exception { String[] parameters = { jobId.toString() }; - int returnCode = frontend.savepoint(parameters); - - assertNotEquals(0, returnCode); - - assertTrue(buffer.toString().contains(expectedTestException)); + try { + frontend.savepoint(parameters); + fail("This should have failed."); + } catch (Exception e) { + assertTrue(ExceptionUtils.findThrowableWithMessage(e, expectedTestException).isPresent()); + } } finally { - clusterClient.shutdown(); restoreStdOutAndStdErr(); } @@ -137,10 +141,13 @@ public void testTriggerSavepointFailureIllegalJobID() throws Exception { new TestingHighAvailabilityServices())); String[] parameters = { "invalid job id" }; - int returnCode = frontend.savepoint(parameters); - assertTrue(buffer.toString().contains("not a valid ID")); - assertNotEquals(0, returnCode); + try { + frontend.savepoint(parameters); + fail("Should have failed."); + } catch (CliArgsException e) { + assertThat(e.getMessage(), Matchers.containsString("Cannot parse JobID")); + } } finally { restoreStdOutAndStdErr(); From 4e4d64b8f323bccda608c21575a7d36c46ba7ad6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 11 Jan 2018 16:44:13 +0100 Subject: [PATCH 12/12] fixup! [FLINK-8333] [flip6] Separate deployment options from command options --- .../java/org/apache/flink/client/cli/CliFrontendCancelTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java index e7d14623a9445..06ac70996d923 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java @@ -57,7 +57,7 @@ public void testCancel() throws Exception { CancelTestCliFrontend testFrontend = new CancelTestCliFrontend(false); int retCode = testFrontend.cancel(parameters); - assertEquals(0,retCode); + assertEquals(0, retCode); Mockito.verify(testFrontend.client, times(1)).cancel(any(JobID.class)); }