Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-11853] [rest] Minimal implementation of POST for /jars/:jarid/plan #7934

Merged
merged 1 commit into from
Mar 21, 2019
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
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,9 @@
import org.apache.flink.runtime.webmonitor.handlers.JarDeleteHeaders;
import org.apache.flink.runtime.webmonitor.handlers.JarListHandler;
import org.apache.flink.runtime.webmonitor.handlers.JarListHeaders;
import org.apache.flink.runtime.webmonitor.handlers.JarPlanGetHeaders;
import org.apache.flink.runtime.webmonitor.handlers.JarPlanHandler;
import org.apache.flink.runtime.webmonitor.handlers.JarPlanHeaders;
import org.apache.flink.runtime.webmonitor.handlers.JarPlanPostHeaders;
import org.apache.flink.runtime.webmonitor.handlers.JarRunHandler;
import org.apache.flink.runtime.webmonitor.handlers.JarRunHeaders;
import org.apache.flink.runtime.webmonitor.handlers.JarUploadHandler;
Expand Down Expand Up @@ -100,7 +101,17 @@ public WebSubmissionExtension(
leaderRetriever,
timeout,
responseHeaders,
JarPlanHeaders.getInstance(),
JarPlanGetHeaders.getInstance(),
jarDir,
configuration,
executor
);

final JarPlanHandler postJarPlanHandler = new JarPlanHandler(
leaderRetriever,
timeout,
responseHeaders,
JarPlanPostHeaders.getInstance(),
jarDir,
configuration,
executor
Expand All @@ -110,7 +121,8 @@ public WebSubmissionExtension(
webSubmissionHandlers.add(Tuple2.of(JarListHeaders.getInstance(), jarListHandler));
webSubmissionHandlers.add(Tuple2.of(JarRunHeaders.getInstance(), jarRunHandler));
webSubmissionHandlers.add(Tuple2.of(JarDeleteHeaders.getInstance(), jarDeleteHandler));
webSubmissionHandlers.add(Tuple2.of(JarPlanHeaders.getInstance(), jarPlanHandler));
webSubmissionHandlers.add(Tuple2.of(JarPlanGetHeaders.getInstance(), jarPlanHandler));
webSubmissionHandlers.add(Tuple2.of(JarPlanGetHeaders.getInstance(), postJarPlanHandler));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.flink.runtime.webmonitor.handlers;

import org.apache.flink.runtime.rest.HttpMethodWrapper;
import org.apache.flink.runtime.rest.messages.JobPlanInfo;
import org.apache.flink.runtime.rest.messages.MessageHeaders;

Expand All @@ -27,9 +26,7 @@
/**
* Message headers for {@link JarPlanHandler}.
*/
public class JarPlanHeaders implements MessageHeaders<JarPlanRequestBody, JobPlanInfo, JarPlanMessageParameters> {

private static final JarPlanHeaders INSTANCE = new JarPlanHeaders();
public abstract class AbstractJarPlanHeaders implements MessageHeaders<JarPlanRequestBody, JobPlanInfo, JarPlanMessageParameters> {

@Override
public Class<JobPlanInfo> getResponseClass() {
Expand All @@ -51,20 +48,11 @@ public JarPlanMessageParameters getUnresolvedMessageParameters() {
return new JarPlanMessageParameters();
}

@Override
public HttpMethodWrapper getHttpMethod() {
return HttpMethodWrapper.GET;
}

@Override
public String getTargetRestEndpointURL() {
return "/jars/:" + JarIdPathParameter.KEY + "/plan";
}

public static JarPlanHeaders getInstance() {
return INSTANCE;
}

@Override
public String getDescription() {
return "Returns the dataflow plan of a job contained in a jar previously uploaded via '" + JarUploadHeaders.URL + "'. " +
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.webmonitor.handlers;

import org.apache.flink.runtime.rest.HttpMethodWrapper;

/**
* Message headers for {@link JarPlanHandler}.
*/
public class JarPlanGetHeaders extends AbstractJarPlanHeaders {

private static final JarPlanGetHeaders INSTANCE = new JarPlanGetHeaders();

@Override
public HttpMethodWrapper getHttpMethod() {
return HttpMethodWrapper.GET;
}

public static JarPlanGetHeaders getInstance() {
return INSTANCE;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.webmonitor.handlers;

import org.apache.flink.runtime.rest.HttpMethodWrapper;

/**
* Message headers for {@link JarPlanHandler}.
*/
public class JarPlanPostHeaders extends AbstractJarPlanHeaders {

private static final JarPlanPostHeaders INSTANCE = new JarPlanPostHeaders();

@Override
public HttpMethodWrapper getHttpMethod() {
return HttpMethodWrapper.POST;
}

public static JarPlanPostHeaders getInstance() {
return INSTANCE;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ public static void setup() throws Exception {
gatewayRetriever,
timeout,
responseHeaders,
JarPlanHeaders.getInstance(),
JarPlanGetHeaders.getInstance(),
jarDir,
new Configuration(),
executor,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ private static JarListInfo listJars(JarListHandler handler, RestfulGateway restf
}

private static JobPlanInfo showPlan(JarPlanHandler handler, String jarName, RestfulGateway restfulGateway) throws Exception {
JarPlanMessageParameters planParameters = JarPlanHeaders.getInstance().getUnresolvedMessageParameters();
JarPlanMessageParameters planParameters = JarPlanGetHeaders.getInstance().getUnresolvedMessageParameters();
HandlerRequest<JarPlanRequestBody, JarPlanMessageParameters> planRequest = new HandlerRequest<>(
new JarPlanRequestBody(),
planParameters,
Expand Down Expand Up @@ -195,7 +195,7 @@ private static class JarHandlers {
gatewayRetriever,
timeout,
responseHeaders,
JarPlanHeaders.getInstance(),
JarPlanGetHeaders.getInstance(),
jarDir,
new Configuration(),
executor);
Expand Down