Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
107 changes: 107 additions & 0 deletions docs/layouts/shortcodes/generated/rest_v1_dispatcher.html
Original file line number Diff line number Diff line change
Expand Up @@ -4252,6 +4252,113 @@
</tr>
</tbody>
</table>
<table class="rest-api table table-bordered">
<tbody>
<tr>
<td class="text-left" colspan="2"><h5><strong>/jobs/:jobid/rescales/summary</strong></h5></td>
</tr>
<tr>
<td class="text-left" style="width: 20%">Verb: <code>GET</code></td>
<td class="text-left">Response code: <code>200 OK</code></td>
</tr>
<tr>
<td colspan="2">Return job rescales summary.</td>
</tr>
<tr>
<td colspan="2">Path parameters</td>
</tr>
<tr>
<td colspan="2">
<ul>
<li><code>jobid</code> - 32-character hexadecimal string value that identifies a job.</li>
</ul>
</td>
</tr>
<tr>
<td colspan="2">
<label>
<details>
<summary>Request</summary>
<pre><code>{}</code></pre>
</label>
</td>
</tr>
<tr>
<td colspan="2">
<label>
<details>
<summary>Response</summary>
<pre><code>{
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:rescales:JobRescalesSummary",
"properties" : {
"completedRescalesDurationStatsInMillis" : {
"type" : "object",
"$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:util:stats:StatsSummaryDto"
},
"failedRescalesDurationStatsInMillis" : {
"type" : "object",
"$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:util:stats:StatsSummaryDto"
},
"ignoredRescalesDurationStatsInMillis" : {
"type" : "object",
"$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:util:stats:StatsSummaryDto"
},
"rescalesCounts" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:rescales:JobRescalesOverview:RescalesCounts",
"properties" : {
"completed" : {
"type" : "integer"
},
"failed" : {
"type" : "integer"
},
"ignored" : {
"type" : "integer"
},
"inProgress" : {
"type" : "integer"
}
}
},
"rescalesDurationStatsInMillis" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:util:stats:StatsSummaryDto",
"properties" : {
"avg" : {
"type" : "integer"
},
"max" : {
"type" : "integer"
},
"min" : {
"type" : "integer"
},
"p50" : {
"type" : "number"
},
"p90" : {
"type" : "number"
},
"p95" : {
"type" : "number"
},
"p99" : {
"type" : "number"
},
"p999" : {
"type" : "number"
}
}
}
}
}</code></pre>
</label>
</td>
</tr>
</tbody>
</table>
<table class="rest-api table table-bordered">
<tbody>
<tr>
Expand Down
31 changes: 31 additions & 0 deletions docs/static/generated/rest_v1_dispatcher.yml
Original file line number Diff line number Diff line change
Expand Up @@ -996,6 +996,24 @@ paths:
application/json:
schema:
$ref: "#/components/schemas/JobRescalesOverview"
/jobs/{jobid}/rescales/summary:
get:
description: Return job rescales summary.
operationId: getJobRescalesSummary
parameters:
- name: jobid
in: path
description: 32-character hexadecimal string value that identifies a job.
required: true
schema:
$ref: "#/components/schemas/JobID"
responses:
"200":
description: The request was successful.
content:
application/json:
schema:
$ref: "#/components/schemas/JobRescalesSummary"
/jobs/{jobid}/rescaling:
patch:
description: Triggers the rescaling of a job. This async operation would return
Expand Down Expand Up @@ -2875,6 +2893,19 @@ components:
$ref: "#/components/schemas/LatestRescales"
rescalesCounts:
$ref: "#/components/schemas/RescalesCounts"
JobRescalesSummary:
type: object
properties:
completedRescalesDurationStatsInMillis:
$ref: "#/components/schemas/StatsSummaryDto"
failedRescalesDurationStatsInMillis:
$ref: "#/components/schemas/StatsSummaryDto"
ignoredRescalesDurationStatsInMillis:
$ref: "#/components/schemas/StatsSummaryDto"
rescalesCounts:
$ref: "#/components/schemas/RescalesCounts"
rescalesDurationStatsInMillis:
$ref: "#/components/schemas/StatsSummaryDto"
JobResourceRequirementsBody:
type: object
additionalProperties:
Expand Down
83 changes: 83 additions & 0 deletions flink-runtime-web/src/test/resources/rest_api_v1.snapshot
Original file line number Diff line number Diff line change
Expand Up @@ -3423,6 +3423,89 @@
}
}
}
}, {
"url" : "/jobs/:jobid/rescales/summary",
"method" : "GET",
"status-code" : "200 OK",
"file-upload" : false,
"path-parameters" : {
"pathParameters" : [ {
"key" : "jobid"
} ]
},
"query-parameters" : {
"queryParameters" : [ ]
},
"request" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyRequestBody"
},
"response" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:rescales:JobRescalesSummary",
"properties" : {
"rescalesCounts" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:rescales:JobRescalesOverview:RescalesCounts",
"properties" : {
"ignored" : {
"type" : "integer"
},
"inProgress" : {
"type" : "integer"
},
"completed" : {
"type" : "integer"
},
"failed" : {
"type" : "integer"
}
}
},
"rescalesDurationStatsInMillis" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:util:stats:StatsSummaryDto",
"properties" : {
"min" : {
"type" : "integer"
},
"max" : {
"type" : "integer"
},
"avg" : {
"type" : "integer"
},
"p50" : {
"type" : "number"
},
"p90" : {
"type" : "number"
},
"p95" : {
"type" : "number"
},
"p99" : {
"type" : "number"
},
"p999" : {
"type" : "number"
}
}
},
"completedRescalesDurationStatsInMillis" : {
"type" : "object",
"$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:util:stats:StatsSummaryDto"
},
"ignoredRescalesDurationStatsInMillis" : {
"type" : "object",
"$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:util:stats:StatsSummaryDto"
},
"failedRescalesDurationStatsInMillis" : {
"type" : "object",
"$ref" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:util:stats:StatsSummaryDto"
}
}
}
}, {
"url" : "/jobs/:jobid/rescaling",
"method" : "PATCH",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,6 @@

package org.apache.flink.runtime.rest.handler.job.rescales;

import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.WebOptions;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.RestHandlerException;
import org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler;
Expand All @@ -38,8 +36,6 @@
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;

import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;

import java.io.IOException;
import java.time.Duration;
import java.util.Collection;
Expand Down Expand Up @@ -80,14 +76,7 @@ private JobRescalesHistory getJobRescalesHistory(ExecutionGraphInfo executionGra
throws RestHandlerException {
if (executionGraphInfo.getRescalesStatsSnapshot() == null
|| executionGraphInfo.getRescalesStatsSnapshot().getRescaleHistory() == null) {
throw new RestHandlerException(
String.format(
"The job `%s` has not enabled the `%s` scheduler, or it has been enabled but the value of configuration option `%s` has not been set to greater than `0`.",
executionGraphInfo.getJobId(),
JobManagerOptions.SchedulerType.Adaptive,
WebOptions.MAX_ADAPTIVE_SCHEDULER_RESCALE_HISTORY_SIZE.key()),
HttpResponseStatus.NOT_FOUND,
RestHandlerException.LoggingBehavior.IGNORE);
throw RescalesUnavailableException.createForJob(executionGraphInfo.getJobId());
}
return JobRescalesHistory.fromRescalesStatsSnapshot(
executionGraphInfo.getRescalesStatsSnapshot());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,6 @@

package org.apache.flink.runtime.rest.handler.job.rescales;

import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.WebOptions;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.RestHandlerException;
import org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler;
Expand All @@ -39,8 +37,6 @@
import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;

import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;

import java.io.IOException;
import java.time.Duration;
import java.util.Collection;
Expand Down Expand Up @@ -81,14 +77,7 @@ private JobRescalesOverview getJobRescalesOverview(ExecutionGraphInfo executionG
throws RestHandlerException {
RescalesStatsSnapshot rescalesStatsSnapshot = executionGraphInfo.getRescalesStatsSnapshot();
if (rescalesStatsSnapshot == null || rescalesStatsSnapshot.getRescaleHistory() == null) {
throw new RestHandlerException(
String.format(
"The job `%s` has not enabled the `%s` scheduler, or it has been enabled but the value of configuration option `%s` has not been set to greater than `0`.",
executionGraphInfo.getJobId(),
JobManagerOptions.SchedulerType.Adaptive,
WebOptions.MAX_ADAPTIVE_SCHEDULER_RESCALE_HISTORY_SIZE.key()),
HttpResponseStatus.NOT_FOUND,
RestHandlerException.LoggingBehavior.IGNORE);
throw RescalesUnavailableException.createForJob(executionGraphInfo.getJobId());
}
return JobRescalesOverview.fromRescalesStatsSnapshot(
executionGraphInfo.getRescalesStatsSnapshot());
Expand Down
Loading