You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2019/05/02 12:02:54 UTC
[flink] branch master updated: [FLINK-12333][docs] Add
documentation for all async operations through REST API
This is an automated email from the ASF dual-hosted git repository.
trohrmann pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 5b42db8 [FLINK-12333][docs] Add documentation for all async operations through REST API
5b42db8 is described below
commit 5b42db8f1338b7677b64ea25609a301d8ba30a81
Author: Yun Tang <my...@live.com>
AuthorDate: Sun Apr 28 21:09:54 2019 +0800
[FLINK-12333][docs] Add documentation for all async operations through REST API
This closes #8327.
---
docs/_includes/generated/rest_v1_dispatcher.html | 8 ++++----
docs/monitoring/rest_api.md | 2 ++
docs/monitoring/rest_api.zh.md | 2 ++
.../async/AsynchronousOperationTriggerMessageHeaders.java | 12 ++++++++++++
.../rest/handler/job/rescaling/RescalingTriggerHeaders.java | 2 +-
.../job/savepoints/SavepointDisposalTriggerHeaders.java | 2 +-
.../messages/job/savepoints/SavepointTriggerHeaders.java | 2 +-
.../job/savepoints/stop/StopWithSavepointTriggerHeaders.java | 4 ++--
.../async/AbstractAsynchronousOperationHandlersTest.java | 5 +++++
9 files changed, 30 insertions(+), 9 deletions(-)
diff --git a/docs/_includes/generated/rest_v1_dispatcher.html b/docs/_includes/generated/rest_v1_dispatcher.html
index 4df103d..b1caac0 100644
--- a/docs/_includes/generated/rest_v1_dispatcher.html
+++ b/docs/_includes/generated/rest_v1_dispatcher.html
@@ -2131,7 +2131,7 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
<td class="text-left">Response code: <code>200 OK</code></td>
</tr>
<tr>
- <td colspan="2">Triggers the rescaling of a job.</td>
+ <td colspan="2">Triggers the rescaling of a job. This async operation would return a 'triggerid' for further query identifier.</td>
</tr>
<tr>
<td colspan="2">Path parameters</td>
@@ -2261,7 +2261,7 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
<td class="text-left">Response code: <code>202 Accepted</code></td>
</tr>
<tr>
- <td colspan="2">Triggers a savepoint, and optionally cancels the job afterwards.</td>
+ <td colspan="2">Triggers a savepoint, and optionally cancels the job afterwards. This async operation would return a 'triggerid' for further query identifier.</td>
</tr>
<tr>
<td colspan="2">Path parameters</td>
@@ -2392,7 +2392,7 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
<td class="text-left">Response code: <code>202 Accepted</code></td>
</tr>
<tr>
- <td colspan="2">Stops a job with a savepoint. Optionally, it can also emit a MAX_WATERMARK before taking the savepoint to flush out any state waiting for timers to fire.</td>
+ <td colspan="2">Stops a job with a savepoint. Optionally, it can also emit a MAX_WATERMARK before taking the savepoint to flush out any state waiting for timers to fire. This async operation would return a 'triggerid' for further query identifier.</td>
</tr>
<tr>
<td colspan="2">Path parameters</td>
@@ -3581,7 +3581,7 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
<td class="text-left">Response code: <code>200 OK</code></td>
</tr>
<tr>
- <td colspan="2">Triggers the desposal of a savepoint.</td>
+ <td colspan="2">Triggers the desposal of a savepoint. This async operation would return a 'triggerid' for further query identifier.</td>
</tr>
<tr>
<td colspan="2">
diff --git a/docs/monitoring/rest_api.md b/docs/monitoring/rest_api.md
index f433163..9c381fd 100644
--- a/docs/monitoring/rest_api.md
+++ b/docs/monitoring/rest_api.md
@@ -61,6 +61,8 @@ If no version is specified Flink will default to the *oldest* version supporting
Querying unsupported/non-existing versions will return a 404 error.
+There exist several async operations among these APIs, e.g. `trigger savepoint`, `rescale a job`. They would return a `triggerid` to identify the operation you just POST and then you need to use that `triggerid` to query for the status of the operation.
+
<div class="codetabs" markdown="1">
<div data-lang="v1" markdown="1">
diff --git a/docs/monitoring/rest_api.zh.md b/docs/monitoring/rest_api.zh.md
index 81fcee6..710b385 100644
--- a/docs/monitoring/rest_api.zh.md
+++ b/docs/monitoring/rest_api.zh.md
@@ -61,6 +61,8 @@ If no version is specified Flink will default to the *oldest* version supporting
Querying unsupported/non-existing versions will return a 404 error.
+There exist several async operations among these APIs, e.g. `trigger savepoint`, `rescale a job`. They would return a `triggerid` to identify the operation you just POST and then you need to use that `triggerid` to query for the status of the operation.
+
<div class="codetabs" markdown="1">
<div data-lang="v1" markdown="1">
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationTriggerMessageHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationTriggerMessageHeaders.java
index 5baa5ad..c641be0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationTriggerMessageHeaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/AsynchronousOperationTriggerMessageHeaders.java
@@ -35,4 +35,16 @@ public abstract class AsynchronousOperationTriggerMessageHeaders<R extends Reque
public Class<TriggerResponse> getResponseClass() {
return TriggerResponse.class;
}
+
+ @Override
+ public String getDescription() {
+ return getAsyncOperationDescription() + " This async operation would return a 'triggerid' for further query identifier.";
+ }
+
+ /**
+ * Returns the description for this async operation header.
+ *
+ * @return the description for this async operation header.
+ */
+ protected abstract String getAsyncOperationDescription();
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingTriggerHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingTriggerHeaders.java
index 86e9c22..dc4977b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingTriggerHeaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingTriggerHeaders.java
@@ -69,7 +69,7 @@ public class RescalingTriggerHeaders extends
}
@Override
- public String getDescription() {
+ protected String getAsyncOperationDescription() {
return "Triggers the rescaling of a job.";
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalTriggerHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalTriggerHeaders.java
index baaf432..dfebabf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalTriggerHeaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalTriggerHeaders.java
@@ -66,7 +66,7 @@ public class SavepointDisposalTriggerHeaders extends AsynchronousOperationTrigge
}
@Override
- public String getDescription() {
+ protected String getAsyncOperationDescription() {
return "Triggers the desposal of a savepoint.";
}
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointTriggerHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointTriggerHeaders.java
index 44400cb..df344b5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointTriggerHeaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointTriggerHeaders.java
@@ -78,7 +78,7 @@ public class SavepointTriggerHeaders
}
@Override
- public String getDescription() {
+ protected String getAsyncOperationDescription() {
return "Triggers a savepoint, and optionally cancels the job afterwards.";
}
}
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 98ff321..3f0bfec 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
@@ -79,8 +79,8 @@ public class StopWithSavepointTriggerHeaders
}
@Override
- public String getDescription() {
+ protected String getAsyncOperationDescription() {
return "Stops a job with a savepoint. Optionally, it can also emit a MAX_WATERMARK before taking" +
- " the savepoint to flush out any state waiting for timers to fire.";
+ " the savepoint to flush out any state waiting for timers to fire.";
}
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/async/AbstractAsynchronousOperationHandlersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/async/AbstractAsynchronousOperationHandlersTest.java
index d414694..f614502 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/async/AbstractAsynchronousOperationHandlersTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/async/AbstractAsynchronousOperationHandlersTest.java
@@ -265,6 +265,11 @@ public class AbstractAsynchronousOperationHandlersTest extends TestLogger {
}
@Override
+ protected String getAsyncOperationDescription() {
+ return "";
+ }
+
+ @Override
public Class<EmptyRequestBody> getRequestClass() {
return EmptyRequestBody.class;
}