Skip to content

Commit

Permalink
[FLINK-28311][rest] Introduce JobManagerEnvironmentHandler
Browse files Browse the repository at this point in the history
  • Loading branch information
KarmaGYZ committed Jul 6, 2022
1 parent ebdde65 commit 1449e8d
Show file tree
Hide file tree
Showing 8 changed files with 557 additions and 0 deletions.
96 changes: 96 additions & 0 deletions docs/layouts/shortcodes/generated/rest_v1_dispatcher.html
Original file line number Diff line number Diff line change
Expand Up @@ -876,6 +876,102 @@
</tr>
</tbody>
</table>
<table class="rest-api table table-bordered">
<tbody>
<tr>
<td class="text-left" colspan="2"><h5><strong>/jobmanager/environment</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 jobmanager environment.</td>
</tr>
<tr>
<td colspan="2">
<div class="book-expand">
<label>
<div class="book-expand-head flex justify-between">
<span>Request</span>
&nbsp; <span></span>
</div>
<input type="checkbox" class="hidden">
<div class="book-expand-content markdown-inner">
<pre>
<code>
{} </code>
</pre>
</div>
</label>
</div>
</td>
</tr>
<tr>
<td colspan="2">
<div class="book-expand">
<label>
<div class="book-expand-head flex justify-between">
<span>Response</span>
&nbsp; <span></span>
</div>
<input type="checkbox" class="hidden">
<div class="book-expand-content markdown-inner">
<pre>
<code>
{
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EnvironmentInfo",
"properties" : {
"classpath" : {
"type" : "array",
"items" : {
"type" : "string"
}
},
"environment" : {
"type" : "array",
"items" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EnvironmentInfo:EnvironmentVariableItem",
"properties" : {
"key" : {
"type" : "string"
},
"value" : {
"type" : "string"
}
}
}
},
"jvm" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EnvironmentInfo:JVMInfo",
"properties" : {
"arch" : {
"type" : "string"
},
"options" : {
"type" : "array",
"items" : {
"type" : "string"
}
},
"version" : {
"type" : "string"
}
}
}
}
} </code>
</pre>
</div>
</label>
</div>
</td>
</tr>
</tbody>
</table>
<table class="rest-api table table-bordered">
<tbody>
<tr>
Expand Down
42 changes: 42 additions & 0 deletions docs/static/generated/rest_v1_dispatcher.yml
Original file line number Diff line number Diff line change
Expand Up @@ -273,6 +273,17 @@ paths:
type: boolean
items:
$ref: '#/components/schemas/ConfigurationInfoEntry'
/jobmanager/environment:
get:
description: Returns the jobmanager environment.
operationId: getJobManagerEnvironment
responses:
"200":
description: The request was successful.
content:
application/json:
schema:
$ref: '#/components/schemas/EnvironmentInfo'
/jobmanager/logs:
get:
description: Returns the list of log files on the JobManager.
Expand Down Expand Up @@ -2021,6 +2032,17 @@ components:
type: object
allOf:
- $ref: '#/components/schemas/SubtaskCheckpointStatistics'
JVMInfo:
type: object
properties:
version:
type: string
arch:
type: string
options:
type: array
items:
type: string
LogInfo:
type: object
properties:
Expand Down Expand Up @@ -2891,6 +2913,19 @@ components:
- FAILED
- CANCELED
- UNKNOWN
EnvironmentInfo:
type: object
properties:
environment:
type: array
items:
$ref: '#/components/schemas/EnvironmentVariableItem'
jvm:
$ref: '#/components/schemas/JVMInfo'
classpath:
type: array
items:
type: string
Summary:
type: object
properties:
Expand Down Expand Up @@ -2918,6 +2953,13 @@ components:
type: string
value:
type: string
EnvironmentVariableItem:
type: object
properties:
key:
type: string
value:
type: string
TriggerResponse:
type: object
properties:
Expand Down
59 changes: 59 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 @@ -483,6 +483,65 @@
}
}
}
}, {
"url" : "/jobmanager/environment",
"method" : "GET",
"status-code" : "200 OK",
"file-upload" : false,
"path-parameters" : {
"pathParameters" : [ ]
},
"query-parameters" : {
"queryParameters" : [ ]
},
"request" : {
"type" : "any"
},
"response" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EnvironmentInfo",
"properties" : {
"environment" : {
"type" : "array",
"items" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EnvironmentInfo:EnvironmentVariableItem",
"properties" : {
"key" : {
"type" : "string"
},
"value" : {
"type" : "string"
}
}
}
},
"jvm" : {
"type" : "object",
"id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EnvironmentInfo:JVMInfo",
"properties" : {
"version" : {
"type" : "string"
},
"arch" : {
"type" : "string"
},
"options" : {
"type" : "array",
"items" : {
"type" : "string"
}
}
}
},
"classpath" : {
"type" : "array",
"items" : {
"type" : "string"
}
}
}
}
}, {
"url" : "/jobmanager/logs",
"method" : "GET",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* 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
*
* https://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.cluster;

import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
import org.apache.flink.runtime.rest.handler.HandlerRequest;
import org.apache.flink.runtime.rest.handler.RestHandlerException;
import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
import org.apache.flink.runtime.rest.messages.EnvironmentInfo;
import org.apache.flink.runtime.rest.messages.MessageHeaders;
import org.apache.flink.runtime.webmonitor.RestfulGateway;
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;

import javax.annotation.Nonnull;

import java.util.Map;
import java.util.concurrent.CompletableFuture;

/** Handler which serves the jobmanager's environment variables. */
public class JobManagerEnvironmentHandler
extends AbstractRestHandler<
RestfulGateway, EmptyRequestBody, EnvironmentInfo, EmptyMessageParameters> {
private final EnvironmentInfo environmentInfo;

public JobManagerEnvironmentHandler(
GatewayRetriever<? extends RestfulGateway> leaderRetriever,
Time timeout,
Map<String, String> responseHeaders,
MessageHeaders<EmptyRequestBody, EnvironmentInfo, EmptyMessageParameters>
messageHeaders) {
super(leaderRetriever, timeout, responseHeaders, messageHeaders);

this.environmentInfo = EnvironmentInfo.create();
}

@Override
protected CompletableFuture<EnvironmentInfo> handleRequest(
@Nonnull HandlerRequest<EmptyRequestBody> request, @Nonnull RestfulGateway gateway)
throws RestHandlerException {
return CompletableFuture.completedFuture(environmentInfo);
}
}
Loading

0 comments on commit 1449e8d

Please sign in to comment.