Skip to content

Commit

Permalink
[FLINK-2111] Add "stop" signal to cleanly shutdown streaming jobs
Browse files Browse the repository at this point in the history
- added JobType to JobGraph and ExecutionGraph
- added interface Stoppable, applied to SourceStreamTask
- added STOP signal logic to JobManager, TaskManager, ExecutionGraph
- extended Client to support stop
- extended Cli frontend, JobManager frontend
- updated documenation

Fix JobManagerTest.testStopSignal and testStopSignalFail

The StoppableInvokable could not be instantiated by Task because it was declared as a private
class. Adds additional checks to verify that the stop signal behaves correctly.

Auto-detect if job is stoppable

A job is stoppable iff all sources are stoppable

- Replace JobType by stoppable flag
- Add StoppableFunction and StoppableInvokable to support the optional stop operation
- added REST get/delete test (no extra YARN test -- think not required as get/delete is both tested)
- bug fix: job got canceld instead of stopped in web interface
- Add StoppingException
- Allow to stop jobs when they are not in state RUNNING

Second round of Till's comments
  • Loading branch information
mjsax authored and tillrohrmann committed Feb 15, 2016
1 parent 5eae47f commit bdd4024
Show file tree
Hide file tree
Showing 53 changed files with 3,389 additions and 1,174 deletions.
14 changes: 14 additions & 0 deletions docs/apis/cli.md
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,10 @@ The command line can be used to

./bin/flink cancel <jobID>

- Stop a job (streaming jobs only):

./bin/flink stop <jobID>

### Savepoints

[Savepoints]({{site.baseurl}}/apis/streaming/savepoints.html) are controlled via the command line client:
Expand Down Expand Up @@ -248,6 +252,16 @@ Action "cancel" cancels a running program.
configuration.
Action "stop" stops a running program (streaming jobs only).
Syntax: stop [OPTIONS] <Job ID>
"stop" action options:
-m,--jobmanager <host:port> Address of the JobManager (master) to which
to connect. Use this flag to connect to a
different JobManager than the one specified
in the configuration.
Action "savepoint" triggers savepoints for a running job or disposes existing ones.
Syntax: savepoint [OPTIONS] <Job ID>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.flink.client.cli.ProgramOptions;
import org.apache.flink.client.cli.RunOptions;
import org.apache.flink.client.cli.SavepointOptions;
import org.apache.flink.client.cli.StopOptions;
import org.apache.flink.client.program.Client;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.client.program.ProgramInvocationException;
Expand All @@ -56,7 +57,10 @@
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
import org.apache.flink.runtime.messages.JobManagerMessages;
import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
import org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure;
import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsStatus;
import org.apache.flink.runtime.messages.JobManagerMessages.StopJob;
import org.apache.flink.runtime.messages.JobManagerMessages.StoppingFailure;
import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
import org.apache.flink.runtime.security.SecurityUtils;
Expand Down Expand Up @@ -110,6 +114,7 @@ public class CliFrontend {
public static final String ACTION_INFO = "info";
private static final String ACTION_LIST = "list";
private static final String ACTION_CANCEL = "cancel";
private static final String ACTION_STOP = "stop";
private static final String ACTION_SAVEPOINT = "savepoint";

// config dir parameters
Expand Down Expand Up @@ -290,9 +295,6 @@ protected int run(String[] args) {
catch (FileNotFoundException e) {
return handleArgException(e);
}
catch (ProgramInvocationException e) {
return handleError(e);
}
catch (Throwable t) {
return handleError(t);
}
Expand Down Expand Up @@ -362,7 +364,7 @@ protected int run(String[] args) {
/**
* Executes the info action.
*
* @param args Command line arguments for the info action.
* @param args Command line arguments for the info action.
*/
protected int info(String[] args) {
LOG.info("Running 'info' command.");
Expand Down Expand Up @@ -567,6 +569,65 @@ public int compare(JobStatusMessage o1, JobStatusMessage o2) {
}
}

/**
* Executes the STOP action.
*
* @param args Command line arguments for the stop action.
*/
protected int stop(String[] args) {
LOG.info("Running 'stop' command.");

StopOptions options;
try {
options = CliFrontendParser.parseStopCommand(args);
}
catch (CliArgsException e) {
return handleArgException(e);
}
catch (Throwable t) {
return handleError(t);
}

// evaluate help flag
if (options.isPrintHelp()) {
CliFrontendParser.printHelpForStop();
return 0;
}

String[] stopArgs = options.getArgs();
JobID jobId;

if (stopArgs.length > 0) {
String jobIdString = stopArgs[0];
try {
jobId = new JobID(StringUtils.hexStringToByte(jobIdString));
}
catch (Exception e) {
return handleError(e);
}
}
else {
return handleArgException(new CliArgsException("Missing JobID"));
}

try {
ActorGateway jobManager = getJobManagerGateway(options);
Future<Object> response = jobManager.ask(new StopJob(jobId), clientTimeout);

final Object rc = Await.result(response, clientTimeout);

if (rc instanceof StoppingFailure) {
throw new Exception("Stopping the job with ID " + jobId + " failed.",
((StoppingFailure) rc).cause());
}

return 0;
}
catch (Throwable t) {
return handleError(t);
}
}

/**
* Executes the CANCEL action.
*
Expand Down Expand Up @@ -616,13 +677,14 @@ protected int cancel(String[] args) {
ActorGateway jobManager = getJobManagerGateway(options);
Future<Object> response = jobManager.ask(new CancelJob(jobId), clientTimeout);

try {
Await.result(response, clientTimeout);
return 0;
}
catch (Exception e) {
throw new Exception("Canceling the job with ID " + jobId + " failed.", e);
final Object rc = Await.result(response, clientTimeout);

if (rc instanceof CancellationFailure) {
throw new Exception("Canceling the job with ID " + jobId + " failed.",
((CancellationFailure) rc).cause());
}

return 0;
}
catch (Throwable t) {
return handleError(t);
Expand Down Expand Up @@ -1123,6 +1185,8 @@ public Integer run() throws Exception {
return info(params);
case ACTION_CANCEL:
return cancel(params);
case ACTION_STOP:
return stop(params);
case ACTION_SAVEPOINT:
return savepoint(params);
case "-h":
Expand All @@ -1139,7 +1203,7 @@ public Integer run() throws Exception {
default:
System.out.printf("\"%s\" is not a valid action.\n", action);
System.out.println();
System.out.println("Valid actions are \"run\", \"list\", \"info\", or \"cancel\".");
System.out.println("Valid actions are \"run\", \"list\", \"info\", \"stop\", or \"cancel\".");
System.out.println();
System.out.println("Specify the version option (-v or --version) to print Flink version.");
System.out.println();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,13 +39,13 @@ public class CliFrontendParser {


static final Option HELP_OPTION = new Option("h", "help", false,
"Show the help message for the CLI Frontend or the action.");
"Show the help message for the CLI Frontend or the action.");

static final Option JAR_OPTION = new Option("j", "jarfile", true, "Flink program JAR file.");

public static final Option CLASS_OPTION = new Option("c", "class", true,
"Class with the program entry point (\"main\" method or \"getPlan()\" method. Only needed if the " +
"JAR file does not specify the class in its manifest.");
"JAR file does not specify the class in its manifest.");

static final Option CLASSPATH_OPTION = new Option("C", "classpath", true, "Adds a URL to each user code " +
"classloader on all nodes in the cluster. The paths must specify a protocol (e.g. file:https://) and be " +
Expand All @@ -55,7 +55,7 @@ public class CliFrontendParser {

static final Option PARALLELISM_OPTION = new Option("p", "parallelism", true,
"The parallelism with which to run the program. Optional flag to override the default value " +
"specified in the configuration.");
"specified in the configuration.");

static final Option LOGGING_OPTION = new Option("q", "sysoutLogging", false, "If present, " +
"supress logging output to standard out.");
Expand All @@ -67,9 +67,9 @@ public class CliFrontendParser {
"Program arguments. Arguments can also be added without -a, simply as trailing parameters.");

static final Option ADDRESS_OPTION = new Option("m", "jobmanager", true,
"Address of the JobManager (master) to which to connect. Specify '" + CliFrontend.YARN_DEPLOY_JOBMANAGER
+ "' as the JobManager to deploy a YARN cluster for the job. Use this flag to connect to a " +
"different JobManager than the one specified in the configuration.");
"Address of the JobManager (master) to which to connect. Specify '" + CliFrontend.YARN_DEPLOY_JOBMANAGER +
"' as the JobManager to deploy a YARN cluster for the job. Use this flag to connect to a " +
"different JobManager than the one specified in the configuration.");

static final Option SAVEPOINT_PATH_OPTION = new Option("s", "fromSavepoint", true,
"Path to a savepoint to reset the job back to (for example file:https:///flink/savepoint-1537).");
Expand Down Expand Up @@ -123,6 +123,7 @@ public class CliFrontendParser {
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 Options buildGeneralOptions(Options options) {
Expand Down Expand Up @@ -197,6 +198,11 @@ private static Options getCancelOptions(Options options) {
return options;
}

private static Options getStopOptions(Options options) {
options = getJobManagerAddressOption(options);
return options;
}

private static Options getSavepointOptions(Options options) {
options = getJobManagerAddressOption(options);
options.addOption(SAVEPOINT_DISPOSE_OPTION);
Expand All @@ -218,6 +224,7 @@ public static void printHelp() {
printHelpForRun();
printHelpForInfo();
printHelpForList();
printHelpForStop();
printHelpForCancel();
printHelpForSavepoint();

Expand Down Expand Up @@ -264,6 +271,18 @@ public static void printHelpForList() {
System.out.println();
}

public static void printHelpForStop() {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);

System.out.println("\nAction \"stop\" stops a running program (streaming jobs only).");
System.out.println("\n Syntax: stop [OPTIONS] <Job ID>");
formatter.setSyntaxPrefix(" \"stop\" action options:");
formatter.printHelp(" ", getStopOptions(new Options()));
System.out.println();
}

public static void printHelpForCancel() {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
Expand Down Expand Up @@ -325,6 +344,16 @@ public static CancelOptions parseCancelCommand(String[] args) throws CliArgsExce
}
}

public static StopOptions parseStopCommand(String[] args) throws CliArgsException {
try {
PosixParser parser = new PosixParser();
CommandLine line = parser.parse(STOP_OPTIONS, args, false);
return new StopOptions(line);
} catch (ParseException e) {
throw new CliArgsException(e.getMessage());
}
}

public static SavepointOptions parseSavepointCommand(String[] args) throws CliArgsException {
try {
PosixParser parser = new PosixParser();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.client.cli;

import org.apache.commons.cli.CommandLine;

/**
* Command line options for the STOP command
*/
public class StopOptions extends CommandLineOptions {

private final String[] args;

public StopOptions(CommandLine line) {
super(line);
this.args = line.getArgs();
}

public String[] getArgs() {
return args == null ? new String[0] : args;
}
}
Loading

0 comments on commit bdd4024

Please sign in to comment.