You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "lining (Jira)" <ji...@apache.org> on 2020/04/03 09:18:00 UTC
[jira] [Updated] (FLINK-14713) Show Historical Attempt For Vertex
SubTask In Rest Api
[ https://issues.apache.org/jira/browse/FLINK-14713?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
lining updated FLINK-14713:
---------------------------
Summary: Show Historical Attempt For Vertex SubTask In Rest Api (was: Show Attempt History in Vertex SubTask In Rest Api)
> Show Historical Attempt For Vertex SubTask In Rest Api
> ------------------------------------------------------
>
> Key: FLINK-14713
> URL: https://issues.apache.org/jira/browse/FLINK-14713
> Project: Flink
> Issue Type: Sub-task
> Components: Runtime / REST
> Reporter: lining
> Priority: Major
>
> Flink jobs could recovery by failover, but the user couldn't see any information about the jobs' failure. There isn't information about the failed attempt.
> h3. Proposed Changes
> h4. Add SubtaskAllExecutionAttemptsDetailsHandler for failed attempt
> * return subtask all attempt and state
> * get prior attempts according to
> {code:java}
> final AccessExecution execution = executionVertex.getCurrentExecutionAttempt();
> final int currentAttemptNum = execution.getAttemptNumber();
> if (currentAttemptNum > 0) {
> for (int i = currentAttemptNum - 1; i >= 0; i--) {
> final AccessExecution currentExecution = executionVertex.getPriorExecutionAttempt(i);
> if (currentExecution != null) {
> allAttempts.add(SubtaskExecutionAttemptDetailsInfo.create(currentExecution, metricFetcher, jobID, jobVertexID));
> }
> }
> }
> {code}
>
> * add SubtaskAllExecutionAttemptsDetailsHandler for prior attempt
> * url /jobs/:jobid/vertices/:vertexid/subtasks/:subtaskIndex/attempts
> * response:
> {code:json}
> {
> "attempts" : {
> "type" : "array",
> "items" : {
> "type" : "object",
> "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:SubtaskExecutionAttemptDetailsInfo",
> "properties" : {
> "subtask" : {
> "type" : "integer"
> },
> "status" : {
> "type" : "string",
> "enum" : [ "CREATED", "SCHEDULED", "DEPLOYING", "RUNNING", "FINISHED", "CANCELING", "CANCELED", "FAILED", "RECONCILING" ]
> },
> "attempt" : {
> "type" : "integer"
> },
> "host" : {
> "type" : "string"
> },
> "start-time" : {
> "type" : "integer"
> },
> "end-time" : {
> "type" : "integer"
> },
> "duration" : {
> "type" : "integer"
> },
> "metrics" : {
> "type" : "object",
> "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:metrics:IOMetricsInfo",
> "properties" : {
> "read-bytes" : {
> "type" : "integer"
> },
> "read-bytes-complete" : {
> "type" : "boolean"
> },
> "write-bytes" : {
> "type" : "integer"
> },
> "write-bytes-complete" : {
> "type" : "boolean"
> },
> "read-records" : {
> "type" : "integer"
> },
> "read-records-complete" : {
> "type" : "boolean"
> },
> "write-records" : {
> "type" : "integer"
> },
> "write-records-complete" : {
> "type" : "boolean"
> }
> }
> }
> }
> }
> }
> }
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)