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
75 changes: 75 additions & 0 deletions docs/layouts/shortcodes/generated/rest_v1_dispatcher.html
Original file line number Diff line number Diff line change
Expand Up @@ -3486,6 +3486,81 @@
</tr>
</tbody>
</table>
<table class="rest-api table table-bordered">
<tbody>
<tr>
<td class="text-left" colspan="2"><h5><strong>/jobs/:jobid/rescales/config</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">Returns the job rescale configuration.</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:JobRescaleConfigInfo",
"properties" : {
"executingCooldownTimeoutInMillis" : {
"type" : "integer"
},
"executingResourceStabilizationTimeoutInMillis" : {
"type" : "integer"
},
"maximumDelayForTriggeringRescaleInMillis" : {
"type" : "integer"
},
"rescaleHistoryMax" : {
"type" : "integer"
},
"rescaleOnFailedCheckpointCount" : {
"type" : "integer"
},
"schedulerExecutionMode" : {
"type" : "string",
"enum" : [ "REACTIVE" ]
},
"slotIdleTimeoutInMillis" : {
"type" : "integer"
},
"submissionResourceStabilizationTimeoutInMillis" : {
"type" : "integer"
},
"submissionResourceWaitTimeoutInMillis" : {
"type" : "integer"
}
}
}</code></pre>
</label>
</td>
</tr>
</tbody>
</table>
<table class="rest-api table table-bordered">
<tbody>
<tr>
Expand Down
51 changes: 51 additions & 0 deletions docs/static/generated/rest_v1_dispatcher.yml
Original file line number Diff line number Diff line change
Expand Up @@ -898,6 +898,24 @@ paths:
application/json:
schema:
$ref: "#/components/schemas/JobPlanInfo"
/jobs/{jobid}/rescales/config:
Comment thread
RocMarshal marked this conversation as resolved.
get:
description: Returns the job rescale configuration.
operationId: getJobRescaleConfig
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/JobRescaleConfigInfo"
/jobs/{jobid}/rescaling:
patch:
description: Triggers the rescaling of a job. This async operation would return
Expand Down Expand Up @@ -2679,6 +2697,35 @@ components:
properties:
plan:
$ref: "#/components/schemas/Plan"
JobRescaleConfigInfo:
type: object
properties:
executingCooldownTimeoutInMillis:
type: integer
format: int64
executingResourceStabilizationTimeoutInMillis:
type: integer
format: int64
maximumDelayForTriggeringRescaleInMillis:
type: integer
format: int64
rescaleHistoryMax:
type: integer
format: int32
rescaleOnFailedCheckpointCount:
type: integer
format: int32
schedulerExecutionMode:
$ref: "#/components/schemas/SchedulerExecutionMode"
slotIdleTimeoutInMillis:
type: integer
format: int64
submissionResourceStabilizationTimeoutInMillis:
type: integer
format: int64
submissionResourceWaitTimeoutInMillis:
type: integer
format: int64
JobResourceRequirementsBody:
type: object
additionalProperties:
Expand Down Expand Up @@ -3071,6 +3118,10 @@ components:
- Default
- Adaptive
- AdaptiveBatch
SchedulerExecutionMode:
type: string
enum:
- REACTIVE
Comment thread
RocMarshal marked this conversation as resolved.
SerializedThrowable:
type: object
properties:
Expand Down
51 changes: 51 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 @@ -4781,5 +4781,56 @@
}
}
}
}, {
"url" : "/jobs/:jobid/rescales/config",
"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:JobRescaleConfigInfo",
"properties" : {
"executingCooldownTimeoutInMillis" : {
"type" : "integer"
},
"executingResourceStabilizationTimeoutInMillis" : {
"type" : "integer"
},
"maximumDelayForTriggeringRescaleInMillis" : {
"type" : "integer"
},
"rescaleHistoryMax" : {
"type" : "integer"
},
"rescaleOnFailedCheckpointCount" : {
"type" : "integer"
},
"schedulerExecutionMode" : {
"type" : "string",
"enum" : [ "REACTIVE" ]
},
"slotIdleTimeoutInMillis" : {
"type" : "integer"
},
"submissionResourceStabilizationTimeoutInMillis" : {
"type" : "integer"
},
"submissionResourceWaitTimeoutInMillis" : {
"type" : "integer"
}
}
}
} ]
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

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

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;
import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.ErrorResponseBody;
import org.apache.flink.runtime.rest.messages.JobIDPathParameter;
import org.apache.flink.runtime.rest.messages.JobMessageParameters;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
import org.apache.flink.runtime.rest.messages.ResponseBody;
import org.apache.flink.runtime.rest.messages.job.rescales.JobRescaleConfigHeaders;
import org.apache.flink.runtime.rest.messages.job.rescales.JobRescaleConfigInfo;
import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
import org.apache.flink.runtime.webmonitor.RestfulGateway;
import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
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;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.Executor;

/** The handler for job rescale configuration. */
public class JobRescaleConfigHandler
extends AbstractExecutionGraphHandler<JobRescaleConfigInfo, JobMessageParameters>
implements JsonArchivist {

public JobRescaleConfigHandler(
GatewayRetriever<? extends RestfulGateway> leaderRetriever,
Duration timeout,
Map<String, String> responseHeaders,
MessageHeaders<EmptyRequestBody, JobRescaleConfigInfo, JobMessageParameters>
messageHeaders,
ExecutionGraphCache executionGraphCache,
Executor executor) {
super(
leaderRetriever,
timeout,
responseHeaders,
messageHeaders,
executionGraphCache,
executor);
}

@Override
protected JobRescaleConfigInfo handleRequest(
HandlerRequest<EmptyRequestBody> request, ExecutionGraphInfo executionGraphInfo)
throws RestHandlerException {
return getJobRescaleConfigInfo(executionGraphInfo);
}

private JobRescaleConfigInfo getJobRescaleConfigInfo(ExecutionGraphInfo executionGraphInfo)
throws RestHandlerException {
if (executionGraphInfo.getJobRescaleConfigInfo() == null) {
throw new RestHandlerException(
"AdaptiveScheduler is not enabled for this job ("
+ executionGraphInfo.getJobId()
+ ").",
HttpResponseStatus.NOT_FOUND,
RestHandlerException.LoggingBehavior.IGNORE);
}
return executionGraphInfo.getJobRescaleConfigInfo();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If executionGraphInfo.getJobRescaleConfigInfo() returns null, throw exception will be a better choice.

}

@Override
public Collection<ArchivedJson> archiveJsonWithPath(ExecutionGraphInfo executionGraphInfo)
throws IOException {

ResponseBody response;
try {
response = getJobRescaleConfigInfo(executionGraphInfo);
} catch (RestHandlerException rhe) {
response = new ErrorResponseBody(rhe.getMessage());
}
return Collections.singletonList(
new ArchivedJson(
JobRescaleConfigHeaders.getInstance()
.getTargetRestEndpointURL()
.replace(
':' + JobIDPathParameter.KEY,
executionGraphInfo.getJobId().toString()),
response));
}
}
Loading