From d329b5b0bdddee1b07b90ff6d6773fe68a1882ac Mon Sep 17 00:00:00 2001 From: maxbelov Date: Mon, 7 May 2018 20:18:21 +0300 Subject: [PATCH] [FLINK-8134] add method getDescription() to MessageHeaders to provide meaningful method description in a generated doc update documentation with this descriptions --- docs/_includes/generated/rest_dispatcher.html | 205 ++++++++++++++---- .../flink/docs/rest/RestAPIDocGenerator.java | 2 +- .../webmonitor/handlers/JarDeleteHeaders.java | 4 + .../webmonitor/handlers/JarListHeaders.java | 5 + .../webmonitor/handlers/JarPlanHeaders.java | 5 + .../webmonitor/handlers/JarRunHeaders.java | 6 + .../webmonitor/handlers/JarUploadHeaders.java | 5 + .../job/rescaling/RescalingStatusHeaders.java | 5 + .../rescaling/RescalingTriggerHeaders.java | 5 + .../rest/messages/BlobServerPortHeaders.java | 5 + .../ClusterConfigurationInfoHeaders.java | 5 + .../rest/messages/ClusterOverviewHeaders.java | 5 + .../DashboardConfigurationHeaders.java | 5 + .../rest/messages/JobAccumulatorsHeaders.java | 5 + .../rest/messages/JobConfigHeaders.java | 5 + .../rest/messages/JobExceptionsHeaders.java | 6 + .../JobIdsWithStatusesOverviewHeaders.java | 5 + .../runtime/rest/messages/JobPlanHeaders.java | 5 + .../rest/messages/JobTerminationHeaders.java | 5 + .../JobVertexAccumulatorsHeaders.java | 5 + .../JobVertexBackPressureHeaders.java | 5 + .../messages/JobVertexDetailsHeaders.java | 5 + .../JobVertexTaskManagersHeaders.java | 6 + .../rest/messages/JobsOverviewHeaders.java | 5 + .../runtime/rest/messages/MessageHeaders.java | 7 + .../SubtasksAllAccumulatorsHandlers.java | 7 + .../rest/messages/SubtasksTimesHeaders.java | 5 + .../checkpoints/CheckpointConfigHeaders.java | 5 + .../CheckpointStatisticDetailsHeaders.java | 5 + .../CheckpointingStatisticsHeaders.java | 5 + .../TaskCheckpointStatisticsHeaders.java | 5 + .../messages/cluster/ShutdownHeaders.java | 5 + .../rest/messages/job/JobDetailsHeaders.java | 6 + .../job/JobExecutionResultHeaders.java | 5 + .../rest/messages/job/JobSubmitHeaders.java | 5 + .../SubtaskCurrentAttemptDetailsHeaders.java | 5 + ...skExecutionAttemptAccumulatorsHeaders.java | 5 + ...SubtaskExecutionAttemptDetailsHeaders.java | 5 + .../metrics/AggregatedJobMetricsHeaders.java | 5 + .../AggregatedSubtaskMetricsHeaders.java | 5 + .../AggregatedTaskManagerMetricsHeaders.java | 6 + .../job/metrics/JobManagerMetricsHeaders.java | 4 + .../job/metrics/JobMetricsHeaders.java | 4 + .../job/metrics/JobVertexMetricsHeaders.java | 4 + .../job/metrics/SubtaskMetricsHeaders.java | 4 + .../metrics/TaskManagerMetricsHeaders.java | 4 + .../SavepointDisposalStatusHeaders.java | 5 + .../SavepointDisposalTriggerHeaders.java | 5 + .../savepoints/SavepointStatusHeaders.java | 5 + .../savepoints/SavepointTriggerHeaders.java | 5 + .../TaskManagerDetailsHeaders.java | 5 + .../taskmanager/TaskManagersHeaders.java | 5 + .../flink/runtime/rest/RestClientTest.java | 5 + .../rest/RestServerEndpointITCase.java | 11 + ...ractAsynchronousOperationHandlersTest.java | 10 + .../metrics/AbstractMetricsHandlerTest.java | 5 + .../metrics/AbstractMetricsHeadersTest.java | 5 + 57 files changed, 448 insertions(+), 48 deletions(-) diff --git a/docs/_includes/generated/rest_dispatcher.html b/docs/_includes/generated/rest_dispatcher.html index 36bf242c652c3..f57a3f3e2d4a6 100644 --- a/docs/_includes/generated/rest_dispatcher.html +++ b/docs/_includes/generated/rest_dispatcher.html @@ -8,7 +8,7 @@ Response code: 200 OK - description + Query the port of blob server. @@ -42,6 +42,42 @@ + + + + + + + + + + + + + + + + + + + +
/cluster
Verb: DELETEResponse code: 200 OK
Shutdown the cluster
+ +
+
+            
+{}            
+          
+
+
+ +
+
+            
+{}            
+          
+
+
@@ -52,7 +88,7 @@ - + - + - + - + @@ -287,7 +323,7 @@ - + @@ -347,7 +383,9 @@ - + @@ -415,7 +453,7 @@ - + - + @@ -513,7 +551,7 @@ - + - + - + @@ -675,7 +713,7 @@ - + - + @@ -880,7 +918,7 @@ - + @@ -936,7 +974,7 @@ - + @@ -1027,7 +1065,7 @@ - + @@ -1330,7 +1368,7 @@ - + @@ -1408,7 +1446,7 @@ - + @@ -1525,7 +1563,7 @@ - + @@ -1671,7 +1709,7 @@ - + @@ -1719,7 +1757,7 @@ - + @@ -1800,7 +1838,7 @@ - + @@ -1866,7 +1904,7 @@ - + @@ -1924,7 +1962,7 @@ - + @@ -1972,7 +2010,7 @@ - + @@ -2036,7 +2074,7 @@ - + @@ -2102,7 +2140,7 @@ - + @@ -2167,7 +2205,7 @@ - + @@ -2233,7 +2271,7 @@ - + @@ -2358,7 +2396,80 @@ - + + + + + + + + + + + + + + + +
Response code: 200 OK
descriptionSome information about the monitoring API and the server setup.
@@ -108,7 +144,7 @@ Response code: 200 OK
descriptionList uploaded jars.
@@ -185,7 +221,7 @@ Response code: 200 OK
descriptionSend a POST request to /jars/upload with your jar file sent as multi-part data under the jarfile file. Also make sure that the multi-part data includes the Content-Type of the file itself, some http libraries do not add the header by default.
@@ -241,7 +277,7 @@ Response code: 200 OK
descriptionDelete an uploaded jar.
Path parametersResponse code: 200 OK
descriptionFetch the plan for a jar.
Path parametersResponse code: 200 OK
descriptionSend a POST request to /jars/:jarid/run. The jarid parameter is the file name of the program JAR in the configured web frontend upload directory (configuration key jobmanager.web.upload.dir). + +If the call succeeds, you will get a response with the ID of the submitted job.
Path parametersResponse code: 200 OK
descriptionGet cluster configuration.
@@ -465,7 +503,7 @@ Response code: 200 OK
descriptionThe metrics for a job manager.
Query parametersResponse code: 200 OK
descriptionAn overview of how many jobs are in which status.
@@ -570,7 +608,7 @@ Response code: 202 Accepted
descriptionSubmit a job to a flink cluster.
@@ -625,7 +663,7 @@ Response code: 200 OK
descriptionThe aggregated metrics for a job.
Query parametersResponse code: 200 OK
descriptionJobs, grouped by status, each with a small summary of its status.
@@ -722,7 +760,7 @@ Response code: 200 OK
descriptionSummary of one job, listing dataflow plan, status, timestamps of state transitions, aggregate information for each vertex (operator).
Path parametersResponse code: 202 Accepted
descriptionCancel the job.
Path parametersResponse code: 200 OK
descriptionGet the aggregated accumulators of a job.
Path parametersResponse code: 200 OK
descriptionGet checkpointing statistics for a given job.
Path parametersResponse code: 200 OK
descriptionGet checkpoint config.
Path parametersResponse code: 200 OK
descriptionGet checkpoint statistic details.
Path parametersResponse code: 200 OK
descriptionTask checkpoint statistics which also includes information about the sub task checkpoint statistics.
Path parametersResponse code: 200 OK
descriptionGet the configuration of a job.
Path parametersResponse code: 200 OK
descriptionThe non-recoverable exceptions that have been observed by the job. The truncated flag defines whether more exceptions occurred, but are not listed, because the response would otherwise get too big.
Path parametersResponse code: 200 OK
descriptionThe result of a job execution. Gives access to the execution time of the job, and to all accumulators created by this job.
Path parametersResponse code: 200 OK
descriptionThe metrics for a specified job.
Path parametersResponse code: 200 OK
descriptionThe dataflow plan of a job. The plan is also included in the job summary (/jobs/).
Path parametersResponse code: 200 OK
descriptionTrigger the rescaling of a job.
Path parametersResponse code: 200 OK
descriptionPolling the status of an ongoing rescaling operation.
Path parametersResponse code: 202 Accepted
descriptionTrigger a savepoint.
Path parametersResponse code: 200 OK
descriptionPolling status for the triggered savepoint.
Path parametersResponse code: 200 OK
descriptionInformation about one specific vertex, with a summary for each of its subtasks.
Path parametersResponse code: 200 OK
descriptionThe aggregated user-defined accumulators, for a specific vertex.
Path parameters
+
    +
  • jobid - description
  • +
  • vertexid - description
  • +
+
+ +
+
+            
+{}            
+          
+
+
+ +
+
+            
+{
+  "type" : "object",
+  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobVertexAccumulatorsInfo",
+  "properties" : {
+    "id" : {
+      "type" : "string"
+    },
+    "user-accumulators" : {
+      "type" : "array",
+      "items" : {
+        "type" : "object",
+        "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:UserAccumulator",
+        "properties" : {
+          "name" : {
+            "type" : "string"
+          },
+          "type" : {
+            "type" : "string"
+          },
+          "value" : {
+            "type" : "string"
+          }
+        }
+      }
+    }
+  }
+}            
+          
+
+
+ + + + + + + + + + + @@ -2431,7 +2542,7 @@ - + @@ -2513,7 +2624,7 @@ - + @@ -2572,7 +2683,7 @@ - + @@ -2633,7 +2744,7 @@ - + @@ -2738,7 +2849,7 @@ - + @@ -2844,7 +2955,7 @@ - + @@ -2925,7 +3036,7 @@ - + @@ -2985,7 +3096,7 @@ - + @@ -3070,7 +3181,7 @@ - + @@ -3192,7 +3303,7 @@ - + - + - + @@ -3377,7 +3488,7 @@ - + - + @@ -3513,7 +3624,7 @@ - + @@ -3666,7 +3777,7 @@ - + diff --git a/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java index 70734da3b0396..de90746034c72 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java @@ -155,7 +155,7 @@ private static String createHtmlEntry(MessageHeaders spec) { sb.append(" \n"); sb.append(" \n"); sb.append(" \n"); - sb.append(" \n"); + sb.append(" \n"); sb.append(" \n"); } if (!pathParameterList.isEmpty()) { diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHeaders.java index bd3cd0a519933..24e8b0e431b6f 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHeaders.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHeaders.java @@ -66,4 +66,8 @@ public static JarDeleteHeaders getInstance() { return INSTANCE; } + @Override + public String getDescription() { + return "Delete an uploaded jar."; + } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHeaders.java index b9036d90fdbf1..349886f0adf31 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHeaders.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHeaders.java @@ -69,4 +69,9 @@ public String getTargetRestEndpointURL() { public static JarListHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "List uploaded jars."; + } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHeaders.java index 1a4544091fd72..f1c35fe136e8e 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHeaders.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHeaders.java @@ -65,4 +65,9 @@ public String getTargetRestEndpointURL() { public static JarPlanHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Fetch the plan for a jar."; + } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java index 0ed035af27d13..42962a63dee8e 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHeaders.java @@ -66,4 +66,10 @@ public String getTargetRestEndpointURL() { public static JarRunHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Send a POST request to /jars/:jarid/run. The jarid parameter is the file name of the program JAR in the configured web frontend upload directory (configuration key jobmanager.web.upload.dir).\n\n" + + "If the call succeeds, you will get a response with the ID of the submitted job."; + } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java index ce6be27638b86..f1df70a4895b1 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHeaders.java @@ -67,4 +67,9 @@ public String getTargetRestEndpointURL() { public static JarUploadHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Send a POST request to /jars/upload with your jar file sent as multi-part data under the jarfile file. Also make sure that the multi-part data includes the Content-Type of the file itself, some http libraries do not add the header by default."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingStatusHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingStatusHeaders.java index 2d5babb17fc7d..57e398576fb16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingStatusHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingStatusHeaders.java @@ -75,4 +75,9 @@ public String getTargetRestEndpointURL() { public static RescalingStatusHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Polling the status of an ongoing rescaling operation."; + } } 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 9567410df342a..72f9d5b9c7050 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 @@ -67,4 +67,9 @@ public String getTargetRestEndpointURL() { public static RescalingTriggerHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Trigger the rescaling of a job."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/BlobServerPortHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/BlobServerPortHeaders.java index 8edec160c4c5d..6daf14f676482 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/BlobServerPortHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/BlobServerPortHeaders.java @@ -66,4 +66,9 @@ public EmptyMessageParameters getUnresolvedMessageParameters() { public static BlobServerPortHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Query the port of blob server."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java index 11a750a08afd1..088b03c61f500 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterConfigurationInfoHeaders.java @@ -69,4 +69,9 @@ public EmptyMessageParameters getUnresolvedMessageParameters() { public static ClusterConfigurationInfoHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get cluster configuration."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterOverviewHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterOverviewHeaders.java index b6ae0f4d9defc..65c58a3634fed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterOverviewHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ClusterOverviewHeaders.java @@ -69,4 +69,9 @@ public EmptyMessageParameters getUnresolvedMessageParameters() { public static ClusterOverviewHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Simple summary of the Flink cluster status."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/DashboardConfigurationHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/DashboardConfigurationHeaders.java index cb14fc515aacb..62076317ff5c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/DashboardConfigurationHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/DashboardConfigurationHeaders.java @@ -66,4 +66,9 @@ public EmptyMessageParameters getUnresolvedMessageParameters() { public static DashboardConfigurationHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Some information about the monitoring API and the server setup."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsHeaders.java index 2e00c91cbb2a5..7f4eb82d34721 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsHeaders.java @@ -70,4 +70,9 @@ public String getTargetRestEndpointURL() { public static JobAccumulatorsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get the aggregated accumulators of a job."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobConfigHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobConfigHeaders.java index fba4f208c4832..eb78225b361a7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobConfigHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobConfigHeaders.java @@ -67,4 +67,9 @@ public String getTargetRestEndpointURL() { public static JobConfigHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get the configuration of a job."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsHeaders.java index 7b924b3b14d84..d06da4333a718 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsHeaders.java @@ -67,4 +67,10 @@ public String getTargetRestEndpointURL() { public static JobExceptionsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "The non-recoverable exceptions that have been observed by the job. The truncated flag defines " + + "whether more exceptions occurred, but are not listed, because the response would otherwise get too big."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobIdsWithStatusesOverviewHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobIdsWithStatusesOverviewHeaders.java index a9af911278ca9..1768cdf025a11 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobIdsWithStatusesOverviewHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobIdsWithStatusesOverviewHeaders.java @@ -67,4 +67,9 @@ public EmptyMessageParameters getUnresolvedMessageParameters() { public static JobIdsWithStatusesOverviewHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "An overview of how many jobs are in which status."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanHeaders.java index 17204bba387c3..6139ea4a0f6a9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobPlanHeaders.java @@ -68,4 +68,9 @@ public String getTargetRestEndpointURL() { public static JobPlanHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "The dataflow plan of a job. The plan is also included in the job summary (/jobs/)."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobTerminationHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobTerminationHeaders.java index a72d4220be43a..c8a3f8a0860e5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobTerminationHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobTerminationHeaders.java @@ -67,4 +67,9 @@ public String getTargetRestEndpointURL() { public static JobTerminationHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Cancel the job."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexAccumulatorsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexAccumulatorsHeaders.java index 350dfc8acfa2a..cf04e340ab292 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexAccumulatorsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexAccumulatorsHeaders.java @@ -71,4 +71,9 @@ public String getTargetRestEndpointURL() { public static JobVertexAccumulatorsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "The aggregated user-defined accumulators, for a specific vertex."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexBackPressureHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexBackPressureHeaders.java index 6ce4bd117c03b..4fa7f2386d886 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexBackPressureHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexBackPressureHeaders.java @@ -65,4 +65,9 @@ public String getTargetRestEndpointURL() { public static JobVertexBackPressureHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get job request backpressure info."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsHeaders.java index ee64333b90b8f..6b7a23a346d31 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsHeaders.java @@ -65,4 +65,9 @@ public String getTargetRestEndpointURL() { public static JobVertexDetailsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Information about one specific vertex, with a summary for each of its subtasks."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersHeaders.java index 84240951772c2..b1d6c9db55e62 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersHeaders.java @@ -71,4 +71,10 @@ public String getTargetRestEndpointURL() { public static JobVertexTaskManagersHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "TaskManager statistics for one specific vertex. This is an aggregation of subtask statistics " + + "returned by /jobs//vertices/."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobsOverviewHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobsOverviewHeaders.java index 5be841cdda540..f048e67c7f52c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobsOverviewHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobsOverviewHeaders.java @@ -69,4 +69,9 @@ public EmptyMessageParameters getUnresolvedMessageParameters() { public static JobsOverviewHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Jobs, grouped by status, each with a small summary of its status."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java index eae70aa36a0d1..535e72970a3a8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageHeaders.java @@ -56,4 +56,11 @@ public interface MessageHeaders> getResponseTypeParameters() { return Collections.emptyList(); } + + /** + * Returns the description for the response type. The description must be valid HTML + * + * @return description for the response type + */ + String getDescription(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksAllAccumulatorsHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksAllAccumulatorsHandlers.java index e178c93aa6c97..cea25b7b113f8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksAllAccumulatorsHandlers.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksAllAccumulatorsHandlers.java @@ -72,4 +72,11 @@ public String getTargetRestEndpointURL() { public static SubtasksAllAccumulatorsHandlers getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Gets all user-defined accumulators for all subtasks of a given vertex. These are the individual " + + "accumulators that are returned in aggregated form by the request " + + "/jobs//vertices//accumulators."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesHeaders.java index 81056da206631..37afaa7174877 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/SubtasksTimesHeaders.java @@ -66,4 +66,9 @@ public String getTargetRestEndpointURL() { public static SubtasksTimesHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get times info for a subtask."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointConfigHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointConfigHeaders.java index f0526a0710d84..8714322395a3c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointConfigHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointConfigHeaders.java @@ -70,4 +70,9 @@ public String getTargetRestEndpointURL() { public static CheckpointConfigHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get checkpoint config."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatisticDetailsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatisticDetailsHeaders.java index efdef40d4964b..7d0e87e22cb5a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatisticDetailsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatisticDetailsHeaders.java @@ -69,4 +69,9 @@ public String getTargetRestEndpointURL() { public static CheckpointStatisticDetailsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get checkpoint statistic details."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatisticsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatisticsHeaders.java index ce809e7ce0c48..7d18affc3ab45 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatisticsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatisticsHeaders.java @@ -68,4 +68,9 @@ public String getTargetRestEndpointURL() { public static CheckpointingStatisticsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get checkpointing statistics for a given job."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/TaskCheckpointStatisticsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/TaskCheckpointStatisticsHeaders.java index 3886b1ff5038b..03adf75ad981b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/TaskCheckpointStatisticsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/TaskCheckpointStatisticsHeaders.java @@ -69,4 +69,9 @@ public String getTargetRestEndpointURL() { public static TaskCheckpointStatisticsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Task checkpoint statistics which also includes information about the sub task checkpoint statistics."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/ShutdownHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/ShutdownHeaders.java index 75a1e99966c52..d80c9c2dd097b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/ShutdownHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/cluster/ShutdownHeaders.java @@ -66,4 +66,9 @@ public String getTargetRestEndpointURL() { public static ShutdownHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Shutdown the cluster"; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsHeaders.java index bd0b7301c43e6..51c0de855494e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsHeaders.java @@ -71,4 +71,10 @@ public String getTargetRestEndpointURL() { public static JobDetailsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Summary of one job, listing dataflow plan, status, timestamps of state transitions, aggregate " + + "information for each vertex (operator)."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExecutionResultHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExecutionResultHeaders.java index a01a2d9274de3..224add7eae691 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExecutionResultHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExecutionResultHeaders.java @@ -68,4 +68,9 @@ public static JobExecutionResultHeaders getInstance() { return INSTANCE; } + @Override + public String getDescription() { + return "The result of a job execution. Gives access to the execution time of the job, " + + "and to all accumulators created by this job."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitHeaders.java index 6235214f80fc6..008a831349db2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitHeaders.java @@ -68,4 +68,9 @@ public EmptyMessageParameters getUnresolvedMessageParameters() { public static JobSubmitHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Submit a job to a flink cluster."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskCurrentAttemptDetailsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskCurrentAttemptDetailsHeaders.java index e64bf3d8aaf2a..4cc15aa9e5922 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskCurrentAttemptDetailsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskCurrentAttemptDetailsHeaders.java @@ -74,4 +74,9 @@ public SubtaskMessageParameters getUnresolvedMessageParameters() { public static SubtaskCurrentAttemptDetailsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Summary of the current or latest execution attempt of a specific subtask."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptAccumulatorsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptAccumulatorsHeaders.java index 5cc159ce89a90..f1514623e684f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptAccumulatorsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptAccumulatorsHeaders.java @@ -75,4 +75,9 @@ public SubtaskAttemptMessageParameters getUnresolvedMessageParameters() { public static SubtaskExecutionAttemptAccumulatorsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "The accumulators collected for one specific subtask during one specific execution attempt (multiple attempts happen in case of failure/recovery)."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsHeaders.java index 6f8eb217220c8..3259967a25cca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/SubtaskExecutionAttemptDetailsHeaders.java @@ -75,4 +75,9 @@ public SubtaskAttemptMessageParameters getUnresolvedMessageParameters() { public static SubtaskExecutionAttemptDetailsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Summary of a specific execution attempt of a specific subtask. Multiple execution attempts happen in case of failure/recovery."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedJobMetricsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedJobMetricsHeaders.java index 265512e1b64f5..c182f85377496 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedJobMetricsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedJobMetricsHeaders.java @@ -41,4 +41,9 @@ public AggregatedJobMetricsParameters getUnresolvedMessageParameters() { public static AggregatedJobMetricsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "The aggregated metrics for a job."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedSubtaskMetricsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedSubtaskMetricsHeaders.java index bfeab5f50dbc3..de2e035e8c58a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedSubtaskMetricsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedSubtaskMetricsHeaders.java @@ -44,4 +44,9 @@ public String getTargetRestEndpointURL() { public static AggregatedSubtaskMetricsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "The aggregated metrics for a vertex subtask."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedTaskManagerMetricsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedTaskManagerMetricsHeaders.java index 5b5fe4c839fed..e5ae2aae4710a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedTaskManagerMetricsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/AggregatedTaskManagerMetricsHeaders.java @@ -41,4 +41,10 @@ public String getTargetRestEndpointURL() { public static AggregatedTaskManagerMetricsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "The aggregated metrics for a task manager."; + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsHeaders.java index f819ff4cf26fe..32d4b48e55553 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsHeaders.java @@ -46,4 +46,8 @@ public static JobManagerMetricsHeaders getInstance() { return INSTANCE; } + @Override + public String getDescription() { + return "The metrics for a job manager."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobMetricsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobMetricsHeaders.java index 393a81edadd65..66528fe577865 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobMetricsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobMetricsHeaders.java @@ -46,4 +46,8 @@ public static JobMetricsHeaders getInstance() { return INSTANCE; } + @Override + public String getDescription() { + return "The metrics for a specified job."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexMetricsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexMetricsHeaders.java index 002b76d5ed422..c7f5d248fd62d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexMetricsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexMetricsHeaders.java @@ -49,4 +49,8 @@ public static JobVertexMetricsHeaders getInstance() { return INSTANCE; } + @Override + public String getDescription() { + return "The metrics for a specified job and job vertex."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/SubtaskMetricsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/SubtaskMetricsHeaders.java index 5471020c66ed3..999352d10e610 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/SubtaskMetricsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/SubtaskMetricsHeaders.java @@ -50,4 +50,8 @@ public static SubtaskMetricsHeaders getInstance() { return INSTANCE; } + @Override + public String getDescription() { + return "Get metrics for a subtask."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsHeaders.java index ddc5fbaf53912..2becb8bd6a9f4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsHeaders.java @@ -47,4 +47,8 @@ public static TaskManagerMetricsHeaders getInstance() { return INSTANCE; } + @Override + public String getDescription() { + return "Get TaskManager metrics."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java index 74deffdc492b0..933896c878440 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java @@ -72,4 +72,9 @@ public static SavepointDisposalStatusHeaders getInstance() { protected Class getValueClass() { return AsynchronousOperationInfo.class; } + + @Override + public String getDescription() { + return "Polling status for the savepoint disposal operation."; + } } 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 5786498505a86..ab2690a3a5230 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 @@ -64,4 +64,9 @@ public String getTargetRestEndpointURL() { public static SavepointDisposalTriggerHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Trigger the disposal of a savepoint."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointStatusHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointStatusHeaders.java index fa012b7c8480b..ea2caeb032386 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointStatusHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointStatusHeaders.java @@ -75,4 +75,9 @@ public static SavepointStatusHeaders getInstance() { protected Class getValueClass() { return SavepointInfo.class; } + + @Override + public String getDescription() { + return "Polling status for the triggered 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 7442bf9179d09..e1984e779a76c 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 @@ -76,4 +76,9 @@ public String getTargetRestEndpointURL() { public static SavepointTriggerHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Trigger a savepoint."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsHeaders.java index 5ff4288786200..c221f338a2fe6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerDetailsHeaders.java @@ -69,4 +69,9 @@ public String getTargetRestEndpointURL() { public static TaskManagerDetailsHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get details for a TaskManager containing base information about executors and detailed metrics."; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagersHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagersHeaders.java index f1e97654a0f37..0f1e6f614bc21 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagersHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagersHeaders.java @@ -70,4 +70,9 @@ public String getTargetRestEndpointURL() { public static TaskManagersHeaders getInstance() { return INSTANCE; } + + @Override + public String getDescription() { + return "Get an overview over all taskmanagers or details for a single one."; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java index eb77af1ed890f..209f2d1d351d4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java @@ -83,6 +83,11 @@ public HttpResponseStatus getResponseStatusCode() { return HttpResponseStatus.OK; } + @Override + public String getDescription() { + return ""; + } + @Override public EmptyMessageParameters getUnresolvedMessageParameters() { return EmptyMessageParameters.getInstance(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java index 09e36de6ecc33..b8bb7f10ca296 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java @@ -493,10 +493,16 @@ public HttpResponseStatus getResponseStatusCode() { return HttpResponseStatus.OK; } + @Override + public String getDescription() { + return ""; + } + @Override public TestParameters getUnresolvedMessageParameters() { return new TestParameters(); } + } private static class TestParameters extends MessageParameters { @@ -626,5 +632,10 @@ public HttpMethodWrapper getHttpMethod() { public String getTargetRestEndpointURL() { return "/upload"; } + + @Override + public String getDescription() { + return ""; + } } } 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 7ad140e6d20bd..d83a5562d26c2 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 @@ -237,6 +237,11 @@ public HttpResponseStatus getResponseStatusCode() { return HttpResponseStatus.OK; } + @Override + public String getDescription() { + return ""; + } + @Override public Class getRequestClass() { return EmptyRequestBody.class; @@ -293,6 +298,11 @@ public HttpMethodWrapper getHttpMethod() { public String getTargetRestEndpointURL() { return "foobar"; } + + @Override + public String getDescription() { + return "description"; + } } private static final class TestingAsynchronousOperationHandlers extends AbstractAsynchronousOperationHandlers { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java index 0d018bc9b76f7..c22b169dab314 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java @@ -221,6 +221,11 @@ public TestMessageParameters getUnresolvedMessageParameters() { public String getTargetRestEndpointURL() { return "/"; } + + @Override + public String getDescription() { + return ""; + } } private static class TestMessageParameters extends MessageParameters { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/AbstractMetricsHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/AbstractMetricsHeadersTest.java index 0ea2d372c1dba..e0adec83dc39a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/AbstractMetricsHeadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/AbstractMetricsHeadersTest.java @@ -53,6 +53,11 @@ public EmptyMessageParameters getUnresolvedMessageParameters() { public String getTargetRestEndpointURL() { return "/"; } + + @Override + public String getDescription() { + return "description"; + } }; }
/jobs/:jobid/vertices/:vertexid/accumulators
Verb: GETResponse code: 200 OK
The aggregated user-defined accumulators, for a specific vertex.
Path parametersResponse code: 200 OK
descriptionGet job request backpressure info.
Path parametersResponse code: 200 OK
descriptionThe metrics for a specified job and job vertex.
Path parametersResponse code: 200 OK
descriptionThe aggregated metrics for a vertex subtask.
Path parametersResponse code: 200 OK
descriptionSummary of the current or latest execution attempt of a specific subtask.
Path parametersResponse code: 200 OK
descriptionSummary of a specific execution attempt of a specific subtask. Multiple execution attempts happen in case of failure/recovery.
Path parametersResponse code: 200 OK
descriptionThe accumulators collected for one specific subtask during one specific execution attempt (multiple attempts happen in case of failure/recovery).
Path parametersResponse code: 200 OK
descriptionGet metrics for a subtask.
Path parametersResponse code: 200 OK
descriptionGet times info for a subtask.
Path parametersResponse code: 200 OK
descriptionTaskManager statistics for one specific vertex. This is an aggregation of subtask statistics returned by /jobs//vertices/.
Path parametersResponse code: 200 OK
descriptionSimple summary of the Flink cluster status.
@@ -3260,7 +3371,7 @@ Response code: 200 OK
descriptionTrigger the disposal of a savepoint.
@@ -3312,7 +3423,7 @@ Response code: 200 OK
descriptionPolling status for the savepoint disposal operation.
Path parametersResponse code: 200 OK
descriptionGet an overview over all taskmanagers or details for a single one.
@@ -3463,7 +3574,7 @@ Response code: 200 OK
descriptionThe aggregated metrics for a task manager.
Query parametersResponse code: 200 OK
descriptionGet details for a TaskManager containing base information about executors and detailed metrics.
Path parametersResponse code: 200 OK
descriptionGet TaskManager metrics.
Path parametersResponse code: " + spec.getResponseStatusCode() + "
" + "description" + "" + spec.getDescription() + "