forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-28311][rest] Introduce JobManagerJobConfigurationHandler
- Loading branch information
Showing
7 changed files
with
378 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
...ain/java/org/apache/flink/runtime/rest/handler/job/JobManagerJobConfigurationHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/* | ||
* 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; | ||
|
||
import org.apache.flink.api.common.time.Time; | ||
import org.apache.flink.configuration.Configuration; | ||
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.ConfigurationInfo; | ||
import org.apache.flink.runtime.rest.messages.EmptyRequestBody; | ||
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.job.JobManagerJobConfigurationHeaders; | ||
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.util.Preconditions; | ||
|
||
import javax.annotation.Nonnull; | ||
|
||
import java.io.IOException; | ||
import java.util.Collection; | ||
import java.util.Collections; | ||
import java.util.Map; | ||
import java.util.concurrent.CompletableFuture; | ||
|
||
/** Handler which serves the jobmanager's configuration of a specific job. */ | ||
public class JobManagerJobConfigurationHandler | ||
extends AbstractRestHandler< | ||
RestfulGateway, EmptyRequestBody, ConfigurationInfo, JobMessageParameters> | ||
implements JsonArchivist { | ||
private final ConfigurationInfo jobConfig; | ||
|
||
public JobManagerJobConfigurationHandler( | ||
GatewayRetriever<? extends RestfulGateway> leaderRetriever, | ||
Time timeout, | ||
Map<String, String> responseHeaders, | ||
MessageHeaders<EmptyRequestBody, ConfigurationInfo, JobMessageParameters> | ||
messageHeaders, | ||
Configuration configuration) { | ||
super(leaderRetriever, timeout, responseHeaders, messageHeaders); | ||
|
||
Preconditions.checkNotNull(configuration); | ||
this.jobConfig = ConfigurationInfo.from(configuration); | ||
} | ||
|
||
@Override | ||
protected CompletableFuture<ConfigurationInfo> handleRequest( | ||
@Nonnull HandlerRequest<EmptyRequestBody> request, @Nonnull RestfulGateway gateway) | ||
throws RestHandlerException { | ||
return CompletableFuture.completedFuture(jobConfig); | ||
} | ||
|
||
@Override | ||
public Collection<ArchivedJson> archiveJsonWithPath(ExecutionGraphInfo executionGraphInfo) | ||
throws IOException { | ||
return Collections.singletonList( | ||
new ArchivedJson( | ||
JobManagerJobConfigurationHeaders.getInstance() | ||
.getTargetRestEndpointURL() | ||
.replace( | ||
':' + JobIDPathParameter.KEY, | ||
executionGraphInfo.getJobId().toString()), | ||
jobConfig)); | ||
} | ||
} |
80 changes: 80 additions & 0 deletions
80
...in/java/org/apache/flink/runtime/rest/messages/job/JobManagerJobConfigurationHeaders.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,80 @@ | ||
/* | ||
* 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.messages.job; | ||
|
||
import org.apache.flink.runtime.rest.HttpMethodWrapper; | ||
import org.apache.flink.runtime.rest.messages.ConfigurationInfo; | ||
import org.apache.flink.runtime.rest.messages.EmptyRequestBody; | ||
import org.apache.flink.runtime.rest.messages.JobMessageParameters; | ||
import org.apache.flink.runtime.rest.messages.MessageHeaders; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
/** | ||
* Message headers for the {@link | ||
* org.apache.flink.runtime.rest.handler.job.JobManagerJobConfigurationHandler}. | ||
*/ | ||
public class JobManagerJobConfigurationHeaders | ||
implements MessageHeaders<EmptyRequestBody, ConfigurationInfo, JobMessageParameters> { | ||
private static final JobManagerJobConfigurationHeaders INSTANCE = | ||
new JobManagerJobConfigurationHeaders(); | ||
|
||
public static final String JOBMANAGER_JOB_CONFIG_REST_PATH = "/jobs/:jobid/jobmanager/config"; | ||
|
||
private JobManagerJobConfigurationHeaders() {} | ||
|
||
@Override | ||
public Class<EmptyRequestBody> getRequestClass() { | ||
return EmptyRequestBody.class; | ||
} | ||
|
||
@Override | ||
public HttpMethodWrapper getHttpMethod() { | ||
return HttpMethodWrapper.GET; | ||
} | ||
|
||
@Override | ||
public String getTargetRestEndpointURL() { | ||
return JOBMANAGER_JOB_CONFIG_REST_PATH; | ||
} | ||
|
||
@Override | ||
public Class<ConfigurationInfo> getResponseClass() { | ||
return ConfigurationInfo.class; | ||
} | ||
|
||
@Override | ||
public HttpResponseStatus getResponseStatusCode() { | ||
return HttpResponseStatus.OK; | ||
} | ||
|
||
@Override | ||
public JobMessageParameters getUnresolvedMessageParameters() { | ||
return new JobMessageParameters(); | ||
} | ||
|
||
public static JobManagerJobConfigurationHeaders getInstance() { | ||
return INSTANCE; | ||
} | ||
|
||
@Override | ||
public String getDescription() { | ||
return "Returns the jobmanager's configuration of a specific job."; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.