Skip to content

Commit

Permalink
[FLINK-12541][REST] Change the rest api from prefix /jars to /artifacts
Browse files Browse the repository at this point in the history
  • Loading branch information
dianfu committed Jun 11, 2019
1 parent 74ef636 commit 2c4d575
Show file tree
Hide file tree
Showing 82 changed files with 4,058 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,17 @@
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.dispatcher.DispatcherGateway;
import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactDeleteHandler;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactDeleteHeaders;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactListHandler;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactListHeaders;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactPlanGetHeaders;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactPlanHandler;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactPlanPostHeaders;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactRunHandler;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactRunHeaders;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactUploadHandler;
import org.apache.flink.runtime.webmonitor.handlers.ArtifactUploadHeaders;
import org.apache.flink.runtime.webmonitor.handlers.JarDeleteHandler;
import org.apache.flink.runtime.webmonitor.handlers.JarDeleteHeaders;
import org.apache.flink.runtime.webmonitor.handlers.JarListHandler;
Expand Down Expand Up @@ -57,18 +68,18 @@ public WebSubmissionExtension(
GatewayRetriever<? extends DispatcherGateway> leaderRetriever,
Map<String, String> responseHeaders,
CompletableFuture<String> localAddressFuture,
Path jarDir,
Path artifactDir,
Executor executor,
Time timeout) throws Exception {

webSubmissionHandlers = new ArrayList<>(5);
webSubmissionHandlers = new ArrayList<>(12);

final JarUploadHandler jarUploadHandler = new JarUploadHandler(
leaderRetriever,
timeout,
responseHeaders,
JarUploadHeaders.getInstance(),
jarDir,
artifactDir,
executor);

final JarListHandler jarListHandler = new JarListHandler(
Expand All @@ -77,15 +88,15 @@ public WebSubmissionExtension(
responseHeaders,
JarListHeaders.getInstance(),
localAddressFuture,
jarDir.toFile(),
artifactDir.toFile(),
executor);

final JarRunHandler jarRunHandler = new JarRunHandler(
leaderRetriever,
timeout,
responseHeaders,
JarRunHeaders.getInstance(),
jarDir,
artifactDir,
configuration,
executor);

Expand All @@ -94,15 +105,15 @@ public WebSubmissionExtension(
timeout,
responseHeaders,
JarDeleteHeaders.getInstance(),
jarDir,
artifactDir,
executor);

final JarPlanHandler jarPlanHandler = new JarPlanHandler(
leaderRetriever,
timeout,
responseHeaders,
JarPlanGetHeaders.getInstance(),
jarDir,
artifactDir,
configuration,
executor
);
Expand All @@ -112,7 +123,61 @@ public WebSubmissionExtension(
timeout,
responseHeaders,
JarPlanPostHeaders.getInstance(),
jarDir,
artifactDir,
configuration,
executor
);

final ArtifactUploadHandler artifactUploadHandler = new ArtifactUploadHandler(
leaderRetriever,
timeout,
responseHeaders,
ArtifactUploadHeaders.getInstance(),
artifactDir,
executor);

final ArtifactListHandler artifactListHandler = new ArtifactListHandler(
leaderRetriever,
timeout,
responseHeaders,
ArtifactListHeaders.getInstance(),
localAddressFuture,
artifactDir.toFile(),
executor);

final ArtifactRunHandler artifactRunHandler = new ArtifactRunHandler(
leaderRetriever,
timeout,
responseHeaders,
ArtifactRunHeaders.getInstance(),
artifactDir,
configuration,
executor);

final ArtifactDeleteHandler artifactDeleteHandler = new ArtifactDeleteHandler(
leaderRetriever,
timeout,
responseHeaders,
ArtifactDeleteHeaders.getInstance(),
artifactDir,
executor);

final ArtifactPlanHandler artifactPlanHandler = new ArtifactPlanHandler(
leaderRetriever,
timeout,
responseHeaders,
ArtifactPlanGetHeaders.getInstance(),
artifactDir,
configuration,
executor
);

final ArtifactPlanHandler postArtifactPlanHandler = new ArtifactPlanHandler(
leaderRetriever,
timeout,
responseHeaders,
ArtifactPlanPostHeaders.getInstance(),
artifactDir,
configuration,
executor
);
Expand All @@ -123,6 +188,13 @@ public WebSubmissionExtension(
webSubmissionHandlers.add(Tuple2.of(JarDeleteHeaders.getInstance(), jarDeleteHandler));
webSubmissionHandlers.add(Tuple2.of(JarPlanGetHeaders.getInstance(), jarPlanHandler));
webSubmissionHandlers.add(Tuple2.of(JarPlanGetHeaders.getInstance(), postJarPlanHandler));

webSubmissionHandlers.add(Tuple2.of(ArtifactUploadHeaders.getInstance(), artifactUploadHandler));
webSubmissionHandlers.add(Tuple2.of(ArtifactListHeaders.getInstance(), artifactListHandler));
webSubmissionHandlers.add(Tuple2.of(ArtifactRunHeaders.getInstance(), artifactRunHandler));
webSubmissionHandlers.add(Tuple2.of(ArtifactDeleteHeaders.getInstance(), artifactDeleteHandler));
webSubmissionHandlers.add(Tuple2.of(ArtifactPlanGetHeaders.getInstance(), artifactPlanHandler));
webSubmissionHandlers.add(Tuple2.of(ArtifactPlanGetHeaders.getInstance(), postArtifactPlanHandler));
}

@Override
Expand Down
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.";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
/**
* Message headers for {@link JarPlanHandler}.
*/
@Deprecated
public abstract class AbstractJarPlanHeaders implements MessageHeaders<JarPlanRequestBody, JobPlanInfo, JarPlanMessageParameters> {

@Override
Expand Down
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);
}
}
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 + "'.";
}
}
Loading

0 comments on commit 2c4d575

Please sign in to comment.