forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-12541][REST] Change the rest api from prefix /jars to /artifacts
- Loading branch information
Showing
82 changed files
with
4,058 additions
and
13 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
63 changes: 63 additions & 0 deletions
63
...c/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractArtifactPlanHeaders.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,63 @@ | ||
/* | ||
* 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.webmonitor.handlers; | ||
|
||
import org.apache.flink.runtime.rest.messages.JobPlanInfo; | ||
import org.apache.flink.runtime.rest.messages.MessageHeaders; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
import static org.apache.flink.runtime.webmonitor.handlers.utils.ArtifactHandlerConstants.ARTIFACT_MESSAGE_HEADER_PREFIX; | ||
|
||
/** | ||
* Message headers for {@link ArtifactPlanHandler}. | ||
*/ | ||
public abstract class AbstractArtifactPlanHeaders implements MessageHeaders<ArtifactPlanRequestBody, JobPlanInfo, ArtifactPlanMessageParameters> { | ||
|
||
@Override | ||
public Class<JobPlanInfo> getResponseClass() { | ||
return JobPlanInfo.class; | ||
} | ||
|
||
@Override | ||
public HttpResponseStatus getResponseStatusCode() { | ||
return HttpResponseStatus.OK; | ||
} | ||
|
||
@Override | ||
public Class<ArtifactPlanRequestBody> getRequestClass() { | ||
return ArtifactPlanRequestBody.class; | ||
} | ||
|
||
@Override | ||
public ArtifactPlanMessageParameters getUnresolvedMessageParameters() { | ||
return new ArtifactPlanMessageParameters(); | ||
} | ||
|
||
@Override | ||
public String getTargetRestEndpointURL() { | ||
return ARTIFACT_MESSAGE_HEADER_PREFIX + "/:" + ArtifactIdPathParameter.KEY + "/plan"; | ||
} | ||
|
||
@Override | ||
public String getDescription() { | ||
return "Returns the dataflow plan of a job contained in an artifact previously uploaded via '" + ArtifactUploadHeaders.URL + "'. " + | ||
"Program arguments can be passed both via the JSON request (recommended) or query parameters."; | ||
} | ||
} |
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
92 changes: 92 additions & 0 deletions
92
...web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ArtifactDeleteHandler.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,92 @@ | ||
/* | ||
* 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.webmonitor.handlers; | ||
|
||
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.EmptyRequestBody; | ||
import org.apache.flink.runtime.rest.messages.EmptyResponseBody; | ||
import org.apache.flink.runtime.rest.messages.MessageHeaders; | ||
import org.apache.flink.runtime.webmonitor.RestfulGateway; | ||
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
import javax.annotation.Nonnull; | ||
|
||
import java.io.IOException; | ||
import java.nio.file.Files; | ||
import java.nio.file.Path; | ||
import java.util.Map; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.CompletionException; | ||
import java.util.concurrent.Executor; | ||
|
||
import static java.util.Objects.requireNonNull; | ||
|
||
/** | ||
* Handles requests for deletion of artifacts. | ||
*/ | ||
public class ArtifactDeleteHandler | ||
extends AbstractRestHandler<RestfulGateway, EmptyRequestBody, EmptyResponseBody, ArtifactDeleteMessageParameters> { | ||
|
||
private final Path artifactDir; | ||
|
||
private final Executor executor; | ||
|
||
public ArtifactDeleteHandler( | ||
final GatewayRetriever<? extends RestfulGateway> leaderRetriever, | ||
final Time timeout, | ||
final Map<String, String> responseHeaders, | ||
final MessageHeaders<EmptyRequestBody, EmptyResponseBody, ArtifactDeleteMessageParameters> messageHeaders, | ||
final Path artifactDir, | ||
final Executor executor) { | ||
super(leaderRetriever, timeout, responseHeaders, messageHeaders); | ||
this.artifactDir = requireNonNull(artifactDir); | ||
this.executor = requireNonNull(executor); | ||
} | ||
|
||
@Override | ||
protected CompletableFuture<EmptyResponseBody> handleRequest( | ||
@Nonnull final HandlerRequest<EmptyRequestBody, ArtifactDeleteMessageParameters> request, | ||
@Nonnull final RestfulGateway gateway) throws RestHandlerException { | ||
|
||
final String artifactId = request.getPathParameter(ArtifactIdPathParameter.class); | ||
return CompletableFuture.supplyAsync(() -> { | ||
final Path artifactToDelete = artifactDir.resolve(artifactId); | ||
if (!Files.exists(artifactToDelete)) { | ||
throw new CompletionException(new RestHandlerException( | ||
String.format("File %s does not exist in %s.", artifactId, artifactDir), | ||
HttpResponseStatus.BAD_REQUEST)); | ||
} else { | ||
try { | ||
Files.delete(artifactToDelete); | ||
return EmptyResponseBody.getInstance(); | ||
} catch (final IOException e) { | ||
throw new CompletionException(new RestHandlerException( | ||
String.format("Failed to delete artifact %s.", artifactToDelete), | ||
HttpResponseStatus.INTERNAL_SERVER_ERROR, | ||
e)); | ||
} | ||
} | ||
}, executor); | ||
} | ||
} |
75 changes: 75 additions & 0 deletions
75
...web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ArtifactDeleteHeaders.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,75 @@ | ||
/* | ||
* 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.webmonitor.handlers; | ||
|
||
import org.apache.flink.runtime.rest.HttpMethodWrapper; | ||
import org.apache.flink.runtime.rest.messages.EmptyRequestBody; | ||
import org.apache.flink.runtime.rest.messages.EmptyResponseBody; | ||
import org.apache.flink.runtime.rest.messages.MessageHeaders; | ||
|
||
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; | ||
|
||
import static org.apache.flink.runtime.webmonitor.handlers.utils.ArtifactHandlerConstants.ARTIFACT_MESSAGE_HEADER_PREFIX; | ||
|
||
/** | ||
* Message headers for {@link ArtifactDeleteHandler}. | ||
*/ | ||
public class ArtifactDeleteHeaders implements MessageHeaders<EmptyRequestBody, EmptyResponseBody, ArtifactDeleteMessageParameters> { | ||
|
||
private static final ArtifactDeleteHeaders INSTANCE = new ArtifactDeleteHeaders(); | ||
|
||
@Override | ||
public Class<EmptyResponseBody> getResponseClass() { | ||
return EmptyResponseBody.class; | ||
} | ||
|
||
@Override | ||
public HttpResponseStatus getResponseStatusCode() { | ||
return HttpResponseStatus.OK; | ||
} | ||
|
||
@Override | ||
public Class<EmptyRequestBody> getRequestClass() { | ||
return EmptyRequestBody.class; | ||
} | ||
|
||
@Override | ||
public ArtifactDeleteMessageParameters getUnresolvedMessageParameters() { | ||
return new ArtifactDeleteMessageParameters(); | ||
} | ||
|
||
@Override | ||
public HttpMethodWrapper getHttpMethod() { | ||
return HttpMethodWrapper.DELETE; | ||
} | ||
|
||
@Override | ||
public String getTargetRestEndpointURL() { | ||
return ARTIFACT_MESSAGE_HEADER_PREFIX + "/:" + ArtifactIdPathParameter.KEY; | ||
} | ||
|
||
public static ArtifactDeleteHeaders getInstance() { | ||
return INSTANCE; | ||
} | ||
|
||
@Override | ||
public String getDescription() { | ||
return "Deletes an artifact previously uploaded via '" + ArtifactUploadHeaders.URL + "'."; | ||
} | ||
} |
Oops, something went wrong.