You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by kk...@apache.org on 2019/07/11 12:01:07 UTC
[flink] 04/05: [FLINK-13123] [rest] align terminology of "stop"
endpoint with cli
This is an automated email from the ASF dual-hosted git repository.
kkloudas pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 8b9ff4f45ea12e0f4e258ce1e152dc2a7a2187cc
Author: Konstantin Knauf <kn...@gmail.com>
AuthorDate: Wed Jul 10 11:07:30 2019 +0200
[FLINK-13123] [rest] align terminology of "stop" endpoint with cli
---
docs/_includes/generated/rest_v1_dispatcher.html | 12 ++++++------
flink-runtime-web/src/test/resources/rest_api_v1.snapshot | 4 ++--
.../runtime/rest/handler/job/JobCancellationHandler.java | 4 +++-
.../rest/handler/job/savepoints/SavepointHandlers.java | 2 +-
.../rest/messages/TerminationModeQueryParameter.java | 4 ++--
.../job/savepoints/stop/StopWithSavepointRequestBody.java | 14 +++++++-------
.../savepoints/stop/StopWithSavepointTriggerHeaders.java | 4 ++--
7 files changed, 23 insertions(+), 21 deletions(-)
diff --git a/docs/_includes/generated/rest_v1_dispatcher.html b/docs/_includes/generated/rest_v1_dispatcher.html
index b1caac0..e3fd943 100644
--- a/docs/_includes/generated/rest_v1_dispatcher.html
+++ b/docs/_includes/generated/rest_v1_dispatcher.html
@@ -2385,7 +2385,7 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
<table class="table table-bordered">
<tbody>
<tr>
- <td class="text-left" colspan="2"><h5><strong>/jobs/:jobid/stop-with-savepoint</strong></h5></td>
+ <td class="text-left" colspan="2"><h5><strong>/jobs/:jobid/stop</strong></h5></td>
</tr>
<tr>
<td class="text-left" style="width: 20%">Verb: <code>POST</code></td>
@@ -2406,8 +2406,8 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
</tr>
<tr>
<td colspan="2">
- <button data-toggle="collapse" data-target="#1622419335">Request</button>
- <div id="1622419335" class="collapse">
+ <button data-toggle="collapse" data-target="#1146617568">Request</button>
+ <div id="1146617568" class="collapse">
<pre>
<code>
{
@@ -2417,7 +2417,7 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
"targetDirectory" : {
"type" : "string"
},
- "endOfEventTime" : {
+ "drain" : {
"type" : "boolean"
}
}
@@ -2428,8 +2428,8 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
</tr>
<tr>
<td colspan="2">
- <button data-toggle="collapse" data-target="#1441715731">Response</button>
- <div id="1441715731" class="collapse">
+ <button data-toggle="collapse" data-target="#965913964">Response</button>
+ <div id="965913964" class="collapse">
<pre>
<code>
{
diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
index 20ef479..e7c4306 100644
--- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
+++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
@@ -1557,7 +1557,7 @@
}
}
}, {
- "url" : "/jobs/:jobid/stop-with-savepoint",
+ "url" : "/jobs/:jobid/stop",
"method" : "POST",
"status-code" : "202 Accepted",
"file-upload" : false,
@@ -1576,7 +1576,7 @@
"targetDirectory" : {
"type" : "string"
},
- "endOfEventTime" : {
+ "drain" : {
"type" : "boolean"
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobCancellationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobCancellationHandler.java
index 62f20a2..48a5f63 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobCancellationHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobCancellationHandler.java
@@ -83,7 +83,9 @@ public class JobCancellationHandler extends AbstractRestHandler<RestfulGateway,
terminationFuture = gateway.cancelJob(jobId, timeout);
break;
case STOP:
- throw new RestHandlerException("The \"stop\" command has been removed. Please use \"stop-with-savepoint\" instead.", HttpResponseStatus.PERMANENT_REDIRECT);
+ throw new RestHandlerException("The termination mode \"stop\" has been removed. For " +
+ "an ungraceful shutdown, please use \"cancel\" instead. For a graceful shutdown, " +
+ "please use \"jobs/:jobId/stop\" instead." , HttpResponseStatus.PERMANENT_REDIRECT);
default:
terminationFuture = FutureUtils.completedExceptionally(new RestHandlerException("Unknown termination mode " + terminationMode + '.', HttpResponseStatus.BAD_REQUEST));
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java
index f6cbd32..68d76fa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java
@@ -150,7 +150,7 @@ public class SavepointHandlers extends AbstractAsynchronousOperationHandlers<Asy
HttpResponseStatus.BAD_REQUEST);
}
- final boolean advanceToEndOfEventTime = request.getRequestBody().shouldAdvanceToEndOfEventTime();
+ final boolean advanceToEndOfEventTime = request.getRequestBody().shouldDrain();
final String targetDirectory = requestedTargetDirectory != null ? requestedTargetDirectory : defaultSavepointDir;
return gateway.stopWithSavepoint(jobId, targetDirectory, advanceToEndOfEventTime, RpcUtils.INF_TIMEOUT);
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java
index 97dadfe..6fb5949 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java
@@ -22,7 +22,7 @@ import org.apache.flink.util.StringUtils;
/**
* Termination mode.
- * @deprecated Only kept to detect legacy usages of the cancel/stop command. Please use the "stop-with-savepoint" command instead.
+ * @deprecated Only kept to detect legacy usages of the cancel/stop command. Please use the "stop" command instead.
*/
public class TerminationModeQueryParameter extends MessageQueryParameter<TerminationModeQueryParameter.TerminationMode> {
@@ -49,7 +49,7 @@ public class TerminationModeQueryParameter extends MessageQueryParameter<Termina
}
/**
- * @deprecated Please use the "stop-with-savepoint" command instead.
+ * @deprecated Please use the "stop" command instead.
*/
public enum TerminationMode {
CANCEL,
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/stop/StopWithSavepointRequestBody.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/stop/StopWithSavepointRequestBody.java
index 785e18b..cd67612 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/stop/StopWithSavepointRequestBody.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/stop/StopWithSavepointRequestBody.java
@@ -32,21 +32,21 @@ public class StopWithSavepointRequestBody implements RequestBody {
public static final String FIELD_NAME_TARGET_DIRECTORY = "targetDirectory";
- private static final String FIELD_NAME_END_OF_EVENT_TIME = "endOfEventTime";
+ private static final String FIELD_NAME_DRAIN = "drain";
@JsonProperty(FIELD_NAME_TARGET_DIRECTORY)
@Nullable
private final String targetDirectory;
- @JsonProperty(FIELD_NAME_END_OF_EVENT_TIME)
- private final boolean endOfEventTime;
+ @JsonProperty(FIELD_NAME_DRAIN)
+ private final boolean drain;
@JsonCreator
public StopWithSavepointRequestBody(
@Nullable @JsonProperty(FIELD_NAME_TARGET_DIRECTORY) final String targetDirectory,
- @JsonProperty(value = FIELD_NAME_END_OF_EVENT_TIME, defaultValue = "false") final boolean endOfEventTime) {
+ @JsonProperty(value = FIELD_NAME_DRAIN, defaultValue = "false") final boolean drain) {
this.targetDirectory = targetDirectory;
- this.endOfEventTime = endOfEventTime;
+ this.drain = drain;
}
@Nullable
@@ -54,7 +54,7 @@ public class StopWithSavepointRequestBody implements RequestBody {
return targetDirectory;
}
- public boolean shouldAdvanceToEndOfEventTime() {
- return endOfEventTime;
+ public boolean shouldDrain() {
+ return drain;
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/stop/StopWithSavepointTriggerHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/stop/StopWithSavepointTriggerHeaders.java
index 3f0bfec..b989d5f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/stop/StopWithSavepointTriggerHeaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/stop/StopWithSavepointTriggerHeaders.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointTriggerMes
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
/**
- * These headers define the protocol for triggering a "stop-with-savepoint" action.
+ * These headers define the protocol for triggering a "stop" action.
*/
public class StopWithSavepointTriggerHeaders
extends AsynchronousOperationTriggerMessageHeaders<StopWithSavepointRequestBody, SavepointTriggerMessageParameters> {
@@ -34,7 +34,7 @@ public class StopWithSavepointTriggerHeaders
private static final StopWithSavepointTriggerHeaders INSTANCE = new StopWithSavepointTriggerHeaders();
private static final String URL = String.format(
- "/jobs/:%s/stop-with-savepoint",
+ "/jobs/:%s/stop",
JobIDPathParameter.KEY);
private StopWithSavepointTriggerHeaders() {