This is an automated email from the ASF dual-hosted git repository.

chesnay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 1e15c4f  [FLINK-11853][rest] Support POST for /job/:jobid/plan
1e15c4f is described below

commit 1e15c4fb2a5c59c99597431d922492d621f9f9aa
Author: Stephen Connolly <[email protected]>
AuthorDate: Thu Mar 7 14:48:18 2019 +0000

    [FLINK-11853][rest] Support POST for /job/:jobid/plan
---
 .../runtime/webmonitor/WebSubmissionExtension.java | 18 ++++++++--
 ...lanHeaders.java => AbstractJarPlanHeaders.java} | 14 +-------
 .../webmonitor/handlers/JarPlanGetHeaders.java     | 38 ++++++++++++++++++++++
 .../webmonitor/handlers/JarPlanPostHeaders.java    | 38 ++++++++++++++++++++++
 .../handlers/JarPlanHandlerParameterTest.java      |  2 +-
 .../webmonitor/handlers/JarSubmissionITCase.java   |  4 +--
 6 files changed, 95 insertions(+), 19 deletions(-)

diff --git 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java
 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java
index ccb2dcb..f145c8a 100644
--- 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java
+++ 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java
@@ -27,8 +27,9 @@ import 
org.apache.flink.runtime.webmonitor.handlers.JarDeleteHandler;
 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;
@@ -100,7 +101,17 @@ public class WebSubmissionExtension implements 
WebMonitorExtension {
                        leaderRetriever,
                        timeout,
                        responseHeaders,
-                       JarPlanHeaders.getInstance(),
+                       JarPlanGetHeaders.getInstance(),
+                       jarDir,
+                       configuration,
+                       executor
+               );
+
+               final JarPlanHandler postJarPlanHandler = new JarPlanHandler(
+                       leaderRetriever,
+                       timeout,
+                       responseHeaders,
+                       JarPlanPostHeaders.getInstance(),
                        jarDir,
                        configuration,
                        executor
@@ -110,7 +121,8 @@ public class WebSubmissionExtension implements 
WebMonitorExtension {
                
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
diff --git 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHeaders.java
 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJarPlanHeaders.java
similarity index 82%
rename from 
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHeaders.java
rename to 
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJarPlanHeaders.java
index 09311d8..157d2db 100644
--- 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHeaders.java
+++ 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJarPlanHeaders.java
@@ -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;
 
@@ -27,9 +26,7 @@ import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseSt
 /**
  * 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() {
@@ -52,19 +49,10 @@ public class JarPlanHeaders implements 
MessageHeaders<JarPlanRequestBody, JobPla
        }
 
        @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 + "'. " +
diff --git 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanGetHeaders.java
 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanGetHeaders.java
new file mode 100644
index 0000000..0ecb907
--- /dev/null
+++ 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanGetHeaders.java
@@ -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;
+       }
+}
diff --git 
a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanPostHeaders.java
 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanPostHeaders.java
new file mode 100644
index 0000000..5bdac94
--- /dev/null
+++ 
b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanPostHeaders.java
@@ -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;
+       }
+}
diff --git 
a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerParameterTest.java
 
b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerParameterTest.java
index 11944d4..90055d9 100644
--- 
a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerParameterTest.java
+++ 
b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerParameterTest.java
@@ -45,7 +45,7 @@ public class JarPlanHandlerParameterTest extends 
JarHandlerParameterTest<JarPlan
                        gatewayRetriever,
                        timeout,
                        responseHeaders,
-                       JarPlanHeaders.getInstance(),
+                       JarPlanGetHeaders.getInstance(),
                        jarDir,
                        new Configuration(),
                        executor,
diff --git 
a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java
 
b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java
index 06728c7..2447a40 100644
--- 
a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java
+++ 
b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java
@@ -126,7 +126,7 @@ public class JarSubmissionITCase extends TestLogger {
        }
 
        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,
@@ -195,7 +195,7 @@ public class JarSubmissionITCase extends TestLogger {
                                gatewayRetriever,
                                timeout,
                                responseHeaders,
-                               JarPlanHeaders.getInstance(),
+                               JarPlanGetHeaders.getInstance(),
                                jarDir,
                                new Configuration(),
                                executor);

Reply via email to