You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by sachingoel0101 <gi...@git.apache.org> on 2015/11/08 14:36:21 UTC

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

GitHub user sachingoel0101 opened a pull request:

    https://github.com/apache/flink/pull/1338

    [FLINK-2978][web-dashboard][webclient] Integrate web client into Web Frontend

    This PR creates an interface on the dashboard to perform the tasks the webclient is doing right now, including displaying plans, submitting jobs and uploading jars.
    
    Here's a quick summary of changes:
    1. GET `/jars/list`: List of all available jars
    2. GET `/jars/:jarid/plan`: Plan for a job. Takes `entry-class`, `parallelism` and `program-args` as query parameters.
    3. DELETE `/jars/:jarid`: Delete a jar.
    4. POST `/jars/upload`: Upload a new jar, limited to 10 MB size. 
    5. POST `/jars/:jarid/run`: Run a jar, and takes arguments just as in GET `/jars/:jarid/plan`.
    
    A new state is introduced on the dashboard, named `submit` which allows all these operations.
    
    Detailed screenshots:
    1. Overview page:
    
    ![overview](https://cloud.githubusercontent.com/assets/8874261/11020609/3ed0b3e4-864a-11e5-8832-a689ac725bcf.png)
    
    2. Disabled mode [via config key `jobmanager.web.submit.allow`], for security purposes:
    
    ![disabled](https://cloud.githubusercontent.com/assets/8874261/11020610/65a29f46-864a-11e5-8fbf-c7d8c05e9803.png)
    
    3. Default state with no jars:
    
    ![no_jars_uploaded](https://cloud.githubusercontent.com/assets/8874261/11020613/71e0b824-864a-11e5-9b53-29d3325f0d65.png)
    
    4. Uploading a jar:
    
    ![uploading](https://cloud.githubusercontent.com/assets/8874261/11020616/7df20802-864a-11e5-8c4b-2fc1bd796e40.png)
    
    5. Arguments passing interface for the programs. This is shown only when a Jar has been selected. At this point, the upload new button disappears, which can be brought back again by unselecting.
    Entry classes can be loaded to the `entry-class` input box by clicking on the list items under Entry Class.
    
    ![argument_passing_interface](https://cloud.githubusercontent.com/assets/8874261/11020622/b1aa81f6-864a-11e5-93ac-100e7fb06564.png)
    
    
    6. Plan, as a result of `Show Plan` button:
    
    ![plan](https://cloud.githubusercontent.com/assets/8874261/11020626/c8e172a8-864a-11e5-9c26-fa12e57edabc.png)
    
    7. Error reporting:
    Invalid entry class:
    
    ![invalid_entry_class](https://cloud.githubusercontent.com/assets/8874261/11020632/fbb36f4c-864a-11e5-98c0-3f98caf8ecfd.png)
    
    Invalid parallelism:
    
    ![invalid_parallelism](https://cloud.githubusercontent.com/assets/8874261/11020633/0d6f2172-864b-11e5-8ac7-b5bfa22db56f.png)
    
    
    8. After clicking on submit, if the job was submitted successfully, it automatically redirects to that job's page.
    
    ![transition_to_job_after_submit](https://cloud.githubusercontent.com/assets/8874261/11020635/291ec12a-864b-11e5-9d69-37dd9de693b0.png)
    
    9. Program arguments:
    
    ![program_args](https://cloud.githubusercontent.com/assets/8874261/11020637/338b30d0-864b-11e5-8797-0e24cc914a93.png)
    
    10. Invalid file uploads:
    
    ![invalid_file_upload](https://cloud.githubusercontent.com/assets/8874261/11020641/447074dc-864b-11e5-907b-7fd80237ee7f.png)
    
    11. Disallow too large files:
    
    ![max_limit_on_upload](https://cloud.githubusercontent.com/assets/8874261/11020644/54b9159c-864b-11e5-90e4-117173ece9a0.png)
    
    
    Of course, the best way to get the feel of it is to use it. :') Looking forward to comments and suggestions. 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/sachingoel0101/flink client

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1338.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1338
    
----
commit 80c538f5a55220a1cf04a8b79f8377fd0a7936b5
Author: Sachin Goel <sa...@gmail.com>
Date:   2015-11-05T05:10:44Z

    [FLINK-2978][web-dashboard][webclient] Integrate web client into Web
    Frontend. Allow security settings to disallow job submission from web.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156503204
  
    Okay, I'll test it again.
    This time, I'm building Flink with Scala 2.11.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155738690
  
    I know. The problem is that MS Azure cloud needs 30 minutes to deploy a small hortonworks cluster. Google handles that in a few minutes.
    But I can use another cluster. The problem is that I'm currently traveling and my connection is not very reliable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45240771
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +/*****************************************************************************
    + * This code is based on the "HttpUploadServerHandler" from the
    + * Netty project's HTTP server example.
    + *
    + * See http://netty.io and
    + * https://github.com/netty/netty/blob/netty-4.0.31.Final/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
    + *****************************************************************************/
    +
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.HttpContent;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpMethod;
    +import io.netty.handler.codec.http.HttpObject;
    +import io.netty.handler.codec.http.HttpRequest;
    +import io.netty.handler.codec.http.LastHttpContent;
    +import io.netty.handler.codec.http.QueryStringDecoder;
    +import io.netty.handler.codec.http.QueryStringEncoder;
    +import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
    +import io.netty.handler.codec.http.multipart.DiskFileUpload;
    +import io.netty.handler.codec.http.multipart.HttpDataFactory;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
    +
    +import java.io.File;
    +import java.util.UUID;
    +
    +/**
    + * Simple code which handles all HTTP requests from the user, and passes them to the Router
    + * handler directly if they do not involve file upload requests.
    + * If a file is required to be uploaded, it handles the upload, and in the http request to the
    + * next handler, passes the name of the file to the next handler.
    + */
    +public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject> {
    +
    +	private HttpRequest request;
    +
    +	private boolean readingChunks;
    +
    +	private static final HttpDataFactory factory = new DefaultHttpDataFactory(true); // use disk
    +
    +	private String requestPath;
    +
    +	private HttpPostRequestDecoder decoder;
    +
    +	private final File uploadDir;
    +
    +	/**
    +	 * The directory where files should be uploaded.
    +	 */
    +	public HttpRequestHandler(File uploadDir) {
    +		this.uploadDir = uploadDir;
    +	}
    +
    +	@Override
    +	public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
    +		if (decoder != null) {
    +			decoder.cleanFiles();
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception {
    +		if (msg instanceof HttpRequest) {
    +			request = (HttpRequest) msg;
    +			requestPath = new QueryStringDecoder(request.getUri()).path();
    +			if (request.getMethod() != HttpMethod.POST) {
    --- End diff --
    
    I'm not very sure about the conventions, but only `PUT` and `POST` methods have a payload associated with them, since they're intuitively *unsafe* methods, which change some state on the server. 
    Further, as for `DELETE`, the HTTP specification states that there is no defined semantics for associating bodies. https://tools.ietf.org/html/rfc7231#section-4.3.5
    Since the netty server currently only processes GET, DELETE and POST, I think we can safely assume no other requests will arrive, and if they do, it'll anyway return a 404.
    
    Your comment however makes me realize that the file upload should happen with a `PUT` request. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155713026
  
    Thank you. I've tested it on a YARN cluster, but the URL it is showing is not correct.
    It seems to me that you are just using the current hostname + ip of the web frontend.
    
    In the cluster, the AM proxy URL is http://hn0-flink0.025yzxu42seezkicpzbkvoljsc.ax.internal.cloudapp.net:8088/proxy/application_1447147086254_0014/#/submit
    
    The Link you are showing is: http://hn0-flink0.025yzxu42seezkicpzbkvoljsc.ax.internal.cloudapp.net:36626/#/submit
    
    the host where the JobManager is running is: wn3-flink0.025yzxu42seezkicpzbkvoljsc.ax.internal.cloudapp.net
    
    I suspect you didn't find the issue because in your testing environment, everything runs on the same host.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155065545
  
    Okay, I see. Lets not fix it as part of this pull request.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156432901
  
    I think we need to improve a bit on the error messages.
    
    This is what I got for submitting a job without arguments on the web interface:
    ```
    Error in invoking the program:
    	org.apache.flink.api.java.ExecutionEnvironment.createProgramPlan(ExecutionEnvironment.java:929)
    	org.apache.flink.api.java.ExecutionEnvironment.createProgramPlan(ExecutionEnvironment.java:908)
    	org.apache.flink.client.program.OptimizerPlanEnvironment.execute(OptimizerPlanEnvironment.java:47)
    	org.apache.flink.api.scala.ExecutionEnvironment.execute(ExecutionEnvironment.scala:627)
    	com.dataartisans.Job$.main(Job.scala:69)
    	com.dataartisans.Job.main(Job.scala)
    	sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    	sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
    	sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    	java.lang.reflect.Method.invoke(Method.java:606)
    	org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:497)
    	org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:395)
    	org.apache.flink.client.program.OptimizerPlanEnvironment.getOptimizedPlan(OptimizerPlanEnvironment.java:80)
    	org.apache.flink.client.program.Client.getOptimizedPlan(Client.java:218)
    	org.apache.flink.runtime.webmonitor.handlers.JarActionHandler.handleRequest(JarActionHandler.java:117)
    
    ```
    
    Starting the same in the IDE will lead to:
    
    ```
    Exception in thread "main" java.lang.RuntimeException: No data for required key 'input'
    	at org.apache.flink.api.java.utils.ParameterTool.getRequired(ParameterTool.java:236)
    	at com.dataartisans.JobWithoutTrigger$.main(JobWithoutTrigger.scala:36)
    	at com.dataartisans.JobWithoutTrigger.main(JobWithoutTrigger.scala)
    	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
    	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    	at java.lang.reflect.Method.invoke(Method.java:497)
    	at com.intellij.rt.execution.application.AppMain.main(AppMain.java:144)
    
    Process finished with exit code 1
    ```
    
    It seems you are cutting away the wrong part of the stack trace 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45088529
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -635,8 +644,18 @@
     	 * The default number of archived jobs for the jobmanager
     	 */
     	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
    -	
    -	
    +
    +	/**
    +	 * By default, submitting jobs from the web-frontend is allowed.
    +	 */
    +	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMISSION = true;
    +
    +	/**
    +	 * Default directory for uploaded file storage for the Web frontend.
    +	 */
    +	public static final String DEFAULT_JOB_MANAGER_WEB_UPLOAD_DIR =
    +			(System.getProperty("java.io.tmpdir") == null ? "/tmp" : System.getProperty("java.io.tmpdir")) + "/webmonitor/";
    --- End diff --
    
    Are we deleting the jar files the user uploaded before stopping the web frontend?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155097754
  
    Since its probably an intentional limitation of the proxy, how about the following approach:
    If the web frontend is running on YARN, we show the direct URL (not using the YARN proxy) in the page of the job submission. I think in most cases, users can directly access the web server on the yarn container.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45088463
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.client.program.PackagedProgram;
    +import org.apache.flink.client.program.ProgramInvocationException;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.Map;
    +import java.util.UUID;
    +import java.util.jar.JarFile;
    +import java.util.jar.Manifest;
    +
    +public class JarListHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public  JarListHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		try {
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			gen.writeStringField("address", params.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY));
    +			if (jarDir != null) {
    --- End diff --
    
    I'm not sure if this is a good way of protecting a security relevant code section.
    Maybe it would make more sense to pass a boolean which controls if we can execute this section?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155059566
  
    Thanks a lot for working on this.
    I've tried out your changes locally, and it was working as expected.
    - Why is there a 10 MB limit on the upload? Usually people use the web interface in their LAN, so big files might be quite common.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45094240
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.StringWriter;
    +import java.util.Map;
    +
    +/**
    + * Handles requests for deletion of jars.
    + */
    +public class JarDeleteHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public JarDeleteHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		final String file = params.get("jarid");
    +		try {
    +			File[] list = jarDir.listFiles(new FilenameFilter() {
    +				@Override
    +				public boolean accept(File dir, String name) {
    +					return name.equals(file);
    +				}
    +			});
    +			boolean success = false;
    +			for (File f: list) {
    +				// although next to impossible for multiple files, we still delete them.
    +				success = success || f.delete();
    +			}
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			if (!success) {
    +				// this seems to always fail on Windows.
    --- End diff --
    
    I think you can close the classloader: http://docs.oracle.com/javase/7/docs/api/java/net/URLClassLoader.html#close()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45102536
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +/*****************************************************************************
    + * This code is based on the "HttpUploadServerHandler" from the
    + * Netty project's HTTP server example.
    + *
    + * See http://netty.io and
    + * https://github.com/netty/netty/blob/netty-4.0.31.Final/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
    + *****************************************************************************/
    +
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.HttpContent;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpMethod;
    +import io.netty.handler.codec.http.HttpObject;
    +import io.netty.handler.codec.http.HttpRequest;
    +import io.netty.handler.codec.http.LastHttpContent;
    +import io.netty.handler.codec.http.QueryStringDecoder;
    +import io.netty.handler.codec.http.QueryStringEncoder;
    +import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
    +import io.netty.handler.codec.http.multipart.DiskFileUpload;
    +import io.netty.handler.codec.http.multipart.HttpDataFactory;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
    +
    +import java.io.File;
    +import java.util.UUID;
    +
    +/**
    + * Simple code which handles all HTTP requests from the user, and passes them to the Router
    + * handler directly if they do not involve file upload requests.
    + * If a file is required to be uploaded, it handles the upload, and in the http request to the
    + * next handler, passes the name of the file to the next handler.
    + */
    +public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject> {
    +
    +	private HttpRequest request;
    +
    +	private boolean readingChunks;
    +
    +	private static final HttpDataFactory factory = new DefaultHttpDataFactory(true); // use disk
    +
    +	private String requestPath;
    +
    +	private HttpPostRequestDecoder decoder;
    +
    +	private final File uploadDir;
    +
    +	/**
    +	 * The directory where files should be uploaded.
    +	 */
    +	public HttpRequestHandler(File uploadDir) {
    +		this.uploadDir = uploadDir;
    +	}
    +
    +	@Override
    +	public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
    +		if (decoder != null) {
    +			decoder.cleanFiles();
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception {
    +		if (msg instanceof HttpRequest) {
    +			request = (HttpRequest) msg;
    +			requestPath = new QueryStringDecoder(request.getUri()).path();
    +			if (request.getMethod() != HttpMethod.POST) {
    --- End diff --
    
    What about DELETE here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45090990
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -635,8 +644,18 @@
     	 * The default number of archived jobs for the jobmanager
     	 */
     	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
    -	
    -	
    +
    +	/**
    +	 * By default, submitting jobs from the web-frontend is allowed.
    +	 */
    +	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMISSION = true;
    +
    +	/**
    +	 * Default directory for uploaded file storage for the Web frontend.
    +	 */
    +	public static final String DEFAULT_JOB_MANAGER_WEB_UPLOAD_DIR =
    +			(System.getProperty("java.io.tmpdir") == null ? "/tmp" : System.getProperty("java.io.tmpdir")) + "/webmonitor/";
    --- End diff --
    
    No. In case user wants them persisted, they should specify a directory themselves. Otherwise, as long as the OS isn't rebooted, the jars will stay there.
    
    Also, perhaps a better idea will be to not have a default directory, and determine it while initializing the Web Monitor. That way, we can get the directory on the job manager machine, instead of the machine where the jar was built.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155084660
  
    @rmetzger. Ah yes. Figured it out. You must be testing on the yarn rm proxy page I assume. Apparently, `POST` requests are not allowed through the proxy to AMs. I forgot to mention this in the description. If you know of a config entry to `yarn-site.xml` which can allow this, please let me know.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45360635
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -635,8 +644,18 @@
     	 * The default number of archived jobs for the jobmanager
     	 */
     	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
    -	
    -	
    +
    +	/**
    +	 * By default, submitting jobs from the web-frontend is allowed.
    +	 */
    +	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMISSION = true;
    +
    +	/**
    +	 * Default directory for uploaded file storage for the Web frontend.
    +	 */
    +	public static final String DEFAULT_JOB_MANAGER_WEB_UPLOAD_DIR =
    +			(System.getProperty("java.io.tmpdir") == null ? "/tmp" : System.getProperty("java.io.tmpdir")) + "/webmonitor/";
    --- End diff --
    
    I like your idea of putting the files into a `/jars/` in the webRootDir, so that the files are deleted on shutdown!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156448110
  
    Hi @rmetzger. Thanks for testing. 
    Error reporting could be improved. I'm working on more detailed stack traces right now.
    
    As for the failing program, is it possible to share the Jar file? I'm parsing similar arguments and they're passed successfully to the program in fact.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45103208
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -302,6 +302,15 @@
     	 */
     	public static final String JOB_MANAGER_WEB_LOG_PATH_KEY = "jobmanager.web.log.path";
     
    +	/**
    +	 * Config parameter indicating whether jobs can be uploaded and run from the web-frontend.
    +	 */
    +	public static final String JOB_MANAGER_WEB_SUBMISSION_KEY = "jobmanager.web.submit.allow";
    --- End diff --
    
    That's a good point. Will take care of this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45091186
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.client.program.PackagedProgram;
    +import org.apache.flink.client.program.ProgramInvocationException;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.Map;
    +import java.util.UUID;
    +import java.util.jar.JarFile;
    +import java.util.jar.Manifest;
    +
    +public class JarListHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public  JarListHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		try {
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			gen.writeStringField("address", params.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY));
    +			if (jarDir != null) {
    --- End diff --
    
    Well, technically, the condition is made sure of the way `jarDir` is initialized in the web monitor. But I can definitely make it more explicit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45245432
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +/*****************************************************************************
    + * This code is based on the "HttpUploadServerHandler" from the
    + * Netty project's HTTP server example.
    + *
    + * See http://netty.io and
    + * https://github.com/netty/netty/blob/netty-4.0.31.Final/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
    + *****************************************************************************/
    +
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.HttpContent;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpMethod;
    +import io.netty.handler.codec.http.HttpObject;
    +import io.netty.handler.codec.http.HttpRequest;
    +import io.netty.handler.codec.http.LastHttpContent;
    +import io.netty.handler.codec.http.QueryStringDecoder;
    +import io.netty.handler.codec.http.QueryStringEncoder;
    +import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
    +import io.netty.handler.codec.http.multipart.DiskFileUpload;
    +import io.netty.handler.codec.http.multipart.HttpDataFactory;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
    +
    +import java.io.File;
    +import java.util.UUID;
    +
    +/**
    + * Simple code which handles all HTTP requests from the user, and passes them to the Router
    + * handler directly if they do not involve file upload requests.
    + * If a file is required to be uploaded, it handles the upload, and in the http request to the
    + * next handler, passes the name of the file to the next handler.
    + */
    +public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject> {
    +
    +	private HttpRequest request;
    +
    +	private boolean readingChunks;
    +
    +	private static final HttpDataFactory factory = new DefaultHttpDataFactory(true); // use disk
    +
    +	private String requestPath;
    +
    +	private HttpPostRequestDecoder decoder;
    +
    +	private final File uploadDir;
    +
    +	/**
    +	 * The directory where files should be uploaded.
    +	 */
    +	public HttpRequestHandler(File uploadDir) {
    +		this.uploadDir = uploadDir;
    +	}
    +
    +	@Override
    +	public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
    +		if (decoder != null) {
    +			decoder.cleanFiles();
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception {
    +		if (msg instanceof HttpRequest) {
    +			request = (HttpRequest) msg;
    +			requestPath = new QueryStringDecoder(request.getUri()).path();
    +			if (request.getMethod() != HttpMethod.POST) {
    --- End diff --
    
    Ah yes. It appears I made a mistake. You're right. `PUT` modifies the state of an existing resource. Just had a more careful look at the rfc too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45087435
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.StringWriter;
    +import java.util.Map;
    +
    +/**
    + * Handles requests for deletion of jars.
    + */
    +public class JarDeleteHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public JarDeleteHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		final String file = params.get("jarid");
    +		try {
    +			File[] list = jarDir.listFiles(new FilenameFilter() {
    +				@Override
    +				public boolean accept(File dir, String name) {
    +					return name.equals(file);
    +				}
    +			});
    +			boolean success = false;
    +			for (File f: list) {
    +				// although next to impossible for multiple files, we still delete them.
    +				success = success || f.delete();
    +			}
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			if (!success) {
    +				// this seems to always fail on Windows.
    --- End diff --
    
    Why is it failing on Windows?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45095325
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.StringWriter;
    +import java.util.Map;
    +
    +/**
    + * Handles requests for deletion of jars.
    + */
    +public class JarDeleteHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public JarDeleteHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		final String file = params.get("jarid");
    +		try {
    +			File[] list = jarDir.listFiles(new FilenameFilter() {
    +				@Override
    +				public boolean accept(File dir, String name) {
    +					return name.equals(file);
    +				}
    +			});
    +			boolean success = false;
    +			for (File f: list) {
    +				// although next to impossible for multiple files, we still delete them.
    +				success = success || f.delete();
    +			}
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			if (!success) {
    +				// this seems to always fail on Windows.
    --- End diff --
    
    Yes. I came across that too. I just gave it somewhat lower preference, since we already have too many issues with deletions on Windows. That's the primary reason `mvn verify` fails.
    Let me try it out and I will report back if it works.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45088017
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/PipelineErrorHandler.java ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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;
    +
    +import com.fasterxml.jackson.core.JsonGenerator;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelHandler;
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.DefaultFullHttpResponse;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpResponseStatus;
    +import io.netty.handler.codec.http.HttpVersion;
    +import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +
    +/**
    + * This is the last handler in the pipeline and logs all error messages.
    + */
    +@ChannelHandler.Sharable
    +public class PipelineErrorHandler extends SimpleChannelInboundHandler<Object> {
    +
    +	@Override
    +	protected void channelRead0(ChannelHandlerContext ctx, Object message) {
    +		// we can't deal with this message. No one in the pipeline handled it. Log it.
    +		System.err.println("Unknown message received: " + message);
    --- End diff --
    
    Why are we not using our regular logging here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155068902
  
    10MB isn't a lot for a fat jar which is how we advise our users to package their programs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45103827
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java ---
    @@ -113,7 +117,17 @@ private void respondAsLeader(ChannelHandlerContext ctx, Routed routed, ActorGate
     		DefaultFullHttpResponse response;
     
     		try {
    -			String result = handler.handleRequest(routed.pathParams(), jobManager);
    +			Map<String, String> params = routed.pathParams();
    --- End diff --
    
    I see, that is a case for query parameters. Is it better to have them all in one map, or to simply give the function two maps?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45090748
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.StringWriter;
    +import java.util.Map;
    +
    +/**
    + * Handles requests for deletion of jars.
    + */
    +public class JarDeleteHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public JarDeleteHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		final String file = params.get("jarid");
    +		try {
    +			File[] list = jarDir.listFiles(new FilenameFilter() {
    +				@Override
    +				public boolean accept(File dir, String name) {
    +					return name.equals(file);
    +				}
    +			});
    +			boolean success = false;
    +			for (File f: list) {
    +				// although next to impossible for multiple files, we still delete them.
    +				success = success || f.delete();
    +			}
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			if (!success) {
    +				// this seems to always fail on Windows.
    --- End diff --
    
    File deletions are an issue on Windows. The problem is that, when a `PackagedProgram` is formed using the jar, and a class loader is constructed, the jars are effectively open. And Windows doesn't allow for deleting them. Linux on the other hand doesn't care.
    This was not working on the earlier webclient either.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156512603
  
    Oh yay. Cool! :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45094279
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.client.program.PackagedProgram;
    +import org.apache.flink.client.program.ProgramInvocationException;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.Map;
    +import java.util.UUID;
    +import java.util.jar.JarFile;
    +import java.util.jar.Manifest;
    +
    +public class JarListHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public  JarListHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		try {
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			gen.writeStringField("address", params.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY));
    +			if (jarDir != null) {
    --- End diff --
    
    That's exactly why I offered to make it more explicit. :')
    On Nov 17, 2015 11:26 PM, "Robert Metzger" <no...@github.com> wrote:
    
    > In
    > flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java
    > <https://github.com/apache/flink/pull/1338#discussion_r45094049>:
    >
    > > +public class JarListHandler implements RequestHandler, RequestHandler.JsonResponse {
    > > +
    > > +	private final File jarDir;
    > > +
    > > +	public  JarListHandler(File jarDirectory) {
    > > +		jarDir = jarDirectory;
    > > +	}
    > > +
    > > +	@Override
    > > +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    > > +		try {
    > > +			StringWriter writer = new StringWriter();
    > > +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    > > +			gen.writeStartObject();
    > > +			gen.writeStringField("address", params.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY));
    > > +			if (jarDir != null) {
    >
    > I don't doubt that its working correctly now. The issue is that its really
    > not obvious what this check is for. Somebody else changing your code might
    > easily break it!
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/flink/pull/1338/files#r45094049>.
    >



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-162334327
  
    I've tested the change again on a cluster, works nicely!
    
    ![image](https://cloud.githubusercontent.com/assets/89049/11614821/5f2f4ca8-9c4e-11e5-87de-65ffa19f2b0e.png)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45244344
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +/*****************************************************************************
    + * This code is based on the "HttpUploadServerHandler" from the
    + * Netty project's HTTP server example.
    + *
    + * See http://netty.io and
    + * https://github.com/netty/netty/blob/netty-4.0.31.Final/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
    + *****************************************************************************/
    +
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.HttpContent;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpMethod;
    +import io.netty.handler.codec.http.HttpObject;
    +import io.netty.handler.codec.http.HttpRequest;
    +import io.netty.handler.codec.http.LastHttpContent;
    +import io.netty.handler.codec.http.QueryStringDecoder;
    +import io.netty.handler.codec.http.QueryStringEncoder;
    +import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
    +import io.netty.handler.codec.http.multipart.DiskFileUpload;
    +import io.netty.handler.codec.http.multipart.HttpDataFactory;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
    +
    +import java.io.File;
    +import java.util.UUID;
    +
    +/**
    + * Simple code which handles all HTTP requests from the user, and passes them to the Router
    + * handler directly if they do not involve file upload requests.
    + * If a file is required to be uploaded, it handles the upload, and in the http request to the
    + * next handler, passes the name of the file to the next handler.
    + */
    +public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject> {
    +
    +	private HttpRequest request;
    +
    +	private boolean readingChunks;
    +
    +	private static final HttpDataFactory factory = new DefaultHttpDataFactory(true); // use disk
    +
    +	private String requestPath;
    +
    +	private HttpPostRequestDecoder decoder;
    +
    +	private final File uploadDir;
    +
    +	/**
    +	 * The directory where files should be uploaded.
    +	 */
    +	public HttpRequestHandler(File uploadDir) {
    +		this.uploadDir = uploadDir;
    +	}
    +
    +	@Override
    +	public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
    +		if (decoder != null) {
    +			decoder.cleanFiles();
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception {
    +		if (msg instanceof HttpRequest) {
    +			request = (HttpRequest) msg;
    +			requestPath = new QueryStringDecoder(request.getUri()).path();
    +			if (request.getMethod() != HttpMethod.POST) {
    --- End diff --
    
    Okay, thanks for clarifying. 
    
    According to this (http://restful-api-design.readthedocs.org/en/latest/methods.html), upload is POST and update is PUT...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45104595
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java ---
    @@ -0,0 +1,253 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.client.program.Client;
    +import org.apache.flink.client.program.PackagedProgram;
    +import org.apache.flink.client.program.ProgramInvocationException;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.optimizer.CompilerException;
    +import org.apache.flink.optimizer.DataStatistics;
    +import org.apache.flink.optimizer.Optimizer;
    +import org.apache.flink.optimizer.costs.DefaultCostEstimator;
    +import org.apache.flink.optimizer.plan.FlinkPlan;
    +import org.apache.flink.optimizer.plan.OptimizedPlan;
    +import org.apache.flink.optimizer.plan.StreamingPlan;
    +import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
    +import org.apache.flink.runtime.client.JobClient;
    +import org.apache.flink.runtime.client.JobExecutionException;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +import org.apache.flink.runtime.jobgraph.JobGraph;
    +import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
    +import scala.concurrent.duration.FiniteDuration;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.io.PrintStream;
    +import java.io.PrintWriter;
    +import java.io.StringWriter;
    +import java.net.URISyntaxException;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * This handler handles requests to fetch plan for an already uploaded jar, as well as for
    + * running it.
    + */
    +public class JarActionHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	private final boolean toRun;
    --- End diff --
    
    Fair enough. Will change this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by gyfora <gi...@git.apache.org>.
Github user gyfora commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-157061461
  
    I will probably do this for future runs. :+1: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156502649
  
    Okay. So I didn't actually cut down on the stack trace. Just made sure that if it's an `ProgramInvocationException` and has an underlying cause, the cause is printed on top, so all the useless trace from netty and web monitor is at the bottom. Otherwise, everything is printed.
    @rmetzger 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155101392
  
    Okay. Cool. 
    
    I think we can post a link to the direct URL in the error message box instead, explaining why it is needed to go to the direct URL.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155064766
  
    This can be fixed, but it requires breaking the existing Netty pipeline. The `HttpObjectAggregator` requires a max content length, which I think cannot be set to too high; 10 MB seemed reasonable. [I tried setting it to 100 MB and the JVM ran out of heap space surprisingly. Does the web monitor also use heap space out of Job Manager? ]
    I am not at all familiar with Netty however, and all I know is what I learnt while working on this. If anyone has an idea how to handle large uploads, in the existing pipeline, please let me know. :)
    Otherwise I think replacing the `HttpObjectAggregator` with something that handles large payloads on disk might be a good idea. An example is here: https://github.com/netty/netty/blob/master/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
    It should be able to handle everything cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45101828
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java ---
    @@ -113,7 +117,17 @@ private void respondAsLeader(ChannelHandlerContext ctx, Routed routed, ActorGate
     		DefaultFullHttpResponse response;
     
     		try {
    -			String result = handler.handleRequest(routed.pathParams(), jobManager);
    +			Map<String, String> params = routed.pathParams();
    --- End diff --
    
    I was not aware of this particular aspect of REST. How should query params be represented? Since here, the resource is `/jars/:jarid`, and representing a plan or submit as `/jars/:jarid/:parallelism/:entry-class/...` doesn't seem okay.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155424425
  
    Okay :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-165752988
  
    I'll merge this PR in the next days. If I find a minor issue on the way, would fix it on the fly.
    
    Thanks for the good work, @sachingoel0101 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156429370
  
    Testing it again on YARN


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by gyfora <gi...@git.apache.org>.
Github user gyfora commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-157045679
  
    I am excited to see this in as well :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45103312
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +/*****************************************************************************
    + * This code is based on the "HttpUploadServerHandler" from the
    + * Netty project's HTTP server example.
    + *
    + * See http://netty.io and
    + * https://github.com/netty/netty/blob/netty-4.0.31.Final/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
    + *****************************************************************************/
    +
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.HttpContent;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpMethod;
    +import io.netty.handler.codec.http.HttpObject;
    +import io.netty.handler.codec.http.HttpRequest;
    +import io.netty.handler.codec.http.LastHttpContent;
    +import io.netty.handler.codec.http.QueryStringDecoder;
    +import io.netty.handler.codec.http.QueryStringEncoder;
    +import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
    +import io.netty.handler.codec.http.multipart.DiskFileUpload;
    +import io.netty.handler.codec.http.multipart.HttpDataFactory;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
    +
    +import java.io.File;
    +import java.util.UUID;
    +
    +/**
    + * Simple code which handles all HTTP requests from the user, and passes them to the Router
    + * handler directly if they do not involve file upload requests.
    + * If a file is required to be uploaded, it handles the upload, and in the http request to the
    + * next handler, passes the name of the file to the next handler.
    + */
    +public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject> {
    +
    +	private HttpRequest request;
    +
    +	private boolean readingChunks;
    +
    +	private static final HttpDataFactory factory = new DefaultHttpDataFactory(true); // use disk
    +
    +	private String requestPath;
    +
    +	private HttpPostRequestDecoder decoder;
    +
    +	private final File uploadDir;
    +
    +	/**
    +	 * The directory where files should be uploaded.
    +	 */
    +	public HttpRequestHandler(File uploadDir) {
    +		this.uploadDir = uploadDir;
    +	}
    +
    +	@Override
    +	public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
    +		if (decoder != null) {
    +			decoder.cleanFiles();
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception {
    +		if (msg instanceof HttpRequest) {
    +			request = (HttpRequest) msg;
    +			requestPath = new QueryStringDecoder(request.getUri()).path();
    +			if (request.getMethod() != HttpMethod.POST) {
    --- End diff --
    
    I'm not sure what you mean.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r46760092
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -302,6 +302,15 @@
     	 */
     	public static final String JOB_MANAGER_WEB_LOG_PATH_KEY = "jobmanager.web.log.path";
     
    +	/**
    +	 * Config parameter indicating whether jobs can be uploaded and run from the web-frontend.
    +	 */
    +	public static final String JOB_MANAGER_WEB_SUBMISSION_KEY = "jobmanager.web.submit.allow";
    --- End diff --
    
    Ah. I think I missed Stephan's first point. Will change this. :smile: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-162349944
  
    @sachingoel0101 , This looks good, let's get it merged soon.
    Some comments on your questions:
    
    1) Concerning the config key: Good that way, thanks!
    
    2) Concerning path parameters vs query parameters: I am not an expert on REST, but I think they serve different roles, so let's keep them separate. I think that two maps are actually not bad. Makes the distinction obvious and is efficient. I am not so convinced of the enum-based approach...
    
    3) Concerning error handling: Let's leave it as it is in this pull request and consolidate it in a separate effort. Should not block this PR:


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156479850
  
    Maybe it makes sense to just show the entire exception to the user, instead of trying to cut away parts of the stack trace?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155074648
  
    Mh, Fabian is right. In practice, jar files very often exceed 10 MB.
    
    >  Does the web monitor also use heap space out of Job Manager? ]
    
    Yes, the web monitor is running in the same JVM as the JM.
    I also have no experience with netty, but it shouldn't be too hard to copy the files to disk without allocating the entire file on the heap first. Can you look into this?
    
    Are you sure you tested the change with YARN?
    I'm getting the following error: http://i.imgur.com/3WDAgfd.png



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-162224088
  
    Hi @rmetzger, can you test this on a cluster again?
    
    @StephanEwen can we discuss your concerns about this? I would like to quickly get this merged after addressing all comments actively as I might not be able to continue working on it after a few weeks. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156483602
  
    Yes. Except for the trace leading up to the handler, I think rest should be okay to show. In case the exception isn't`ProgramInvocationException` or `CompilerException`, something went really wrong anyway, in which case there's no harm printing the entire stack trace.
    Fixing it right now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156494598
  
    Okay, let me know when you are ready for another test drive


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156510858
  
    Good news. Test on cluster was successful.
    I'll take another look at the code, but otherwise +1 from my side.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45100395
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java ---
    @@ -113,7 +117,17 @@ private void respondAsLeader(ChannelHandlerContext ctx, Routed routed, ActorGate
     		DefaultFullHttpResponse response;
     
     		try {
    -			String result = handler.handleRequest(routed.pathParams(), jobManager);
    +			Map<String, String> params = routed.pathParams();
    --- End diff --
    
    If you pass two maps (routed + query) then this is much less effort per request, and keeps the types of parameters nicely separated.
    
    Also: Why do you need the query parameter here? This handler is the base for REST style calls, it should never need query parameters.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155726578
  
    @rmetzger I have pushed a fix. Can you check this again? I apologize if this takes one two more iterations to get right since I cannot check this myself.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156433498
  
    Even with all the fields filled, it was not starting: http://i.imgur.com/5WXKw9N.png
    
    the log file also contained no helpful information


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45102455
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -302,6 +302,15 @@
     	 */
     	public static final String JOB_MANAGER_WEB_LOG_PATH_KEY = "jobmanager.web.log.path";
     
    +	/**
    +	 * Config parameter indicating whether jobs can be uploaded and run from the web-frontend.
    +	 */
    +	public static final String JOB_MANAGER_WEB_SUBMISSION_KEY = "jobmanager.web.submit.allow";
    --- End diff --
    
    Think of a config as a JSON object. There is an object "jobmanager.web" and it contains an object "submit" with a field "allow."
    
    1) I think "enable" is better than "allow".
    2) You have two different nested objects, called "submit" and "upload". Since they refer to the same function, makes sense to fuse them, in my opinion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45087911
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -635,8 +644,18 @@
     	 * The default number of archived jobs for the jobmanager
     	 */
     	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
    -	
    -	
    +
    +	/**
    +	 * By default, submitting jobs from the web-frontend is allowed.
    +	 */
    +	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMISSION = true;
    +
    +	/**
    +	 * Default directory for uploaded file storage for the Web frontend.
    +	 */
    +	public static final String DEFAULT_JOB_MANAGER_WEB_UPLOAD_DIR =
    +			(System.getProperty("java.io.tmpdir") == null ? "/tmp" : System.getProperty("java.io.tmpdir")) + "/webmonitor/";
    --- End diff --
    
    I'm not sure if this is the best way of doing this. Is `/tmp` also always the default directory for Flink on YARN?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156477959
  
    Yes. I will write some better exception reporting. As for this particular error, how can it be detected? 
    From what I observed, ideally, `Client#getOptimizedPlan` sets `OptimizerPlanEnvironment` which in turn constructs an `StreamPlanEnvironment`. But when the scala version doesn't match, the `execute` call delegates to the default `ExecutionEnvironment#execute` instead of `StreamPlanEnvironment#execute`, which of course leads to error since yours was a Streaming program.  Can't quite see why that should happen.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-164384200
  
    @StephanEwen, @rmetzger. Ping. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155732014
  
    Thank you. Sadly, I've shut down the cluster a few minutes ago, because I was finished with the release testing.
    I'll try soon...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-160656791
  
    Thank you for fixing the upload directory issue.
    The URLClassloader is really not a blocker, no.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45094049
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.client.program.PackagedProgram;
    +import org.apache.flink.client.program.ProgramInvocationException;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.Map;
    +import java.util.UUID;
    +import java.util.jar.JarFile;
    +import java.util.jar.Manifest;
    +
    +public class JarListHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public  JarListHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		try {
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			gen.writeStringField("address", params.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY));
    +			if (jarDir != null) {
    --- End diff --
    
    I don't doubt that its working correctly now. The issue is that its really not obvious what this check is for. Somebody else changing your code might easily break it!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-157471191
  
    The screenshots look very nice, this will be a good feature!
    
    A few things in the internals could be cleaned up:
      - Mixing of path and query params into one map
      - Putting endpoint address into the map
      - Error handling seems to always give a "Bad Request" error code. Would be good to react to that properly, return a "Not Fount" or "Already Exists", or whatever, specific to the situation. This may require to create an Exception class (RequestException) that contains the response message and desired return code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155423576
  
    Not yet. I'm still fine-tuning some things. Will ping you when it's ready.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45317406
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -635,8 +644,18 @@
     	 * The default number of archived jobs for the jobmanager
     	 */
     	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
    -	
    -	
    +
    +	/**
    +	 * By default, submitting jobs from the web-frontend is allowed.
    +	 */
    +	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMISSION = true;
    +
    +	/**
    +	 * Default directory for uploaded file storage for the Web frontend.
    +	 */
    +	public static final String DEFAULT_JOB_MANAGER_WEB_UPLOAD_DIR =
    +			(System.getProperty("java.io.tmpdir") == null ? "/tmp" : System.getProperty("java.io.tmpdir")) + "/webmonitor/";
    --- End diff --
    
    @rmetzger can we make a decision on this? I'm in favor of having a directory `/jars/` inside the `webRootDir` created by the Web Monitor. There is already a shutdown hook for removing it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45094375
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.client.program.PackagedProgram;
    +import org.apache.flink.client.program.ProgramInvocationException;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler;
    +
    +import java.io.File;
    +import java.io.FilenameFilter;
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.Map;
    +import java.util.UUID;
    +import java.util.jar.JarFile;
    +import java.util.jar.Manifest;
    +
    +public class JarListHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	public  JarListHandler(File jarDirectory) {
    +		jarDir = jarDirectory;
    +	}
    +
    +	@Override
    +	public String handleRequest(Map<String, String> params, ActorGateway jobManager) throws Exception {
    +		try {
    +			StringWriter writer = new StringWriter();
    +			JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
    +			gen.writeStartObject();
    +			gen.writeStringField("address", params.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY));
    +			if (jarDir != null) {
    --- End diff --
    
    Sorry, You are right, there was no need to justify myself ;)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155421694
  
    I assume the PR is ready for another round of reviewing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r46153734
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -302,6 +302,15 @@
     	 */
     	public static final String JOB_MANAGER_WEB_LOG_PATH_KEY = "jobmanager.web.log.path";
     
    +	/**
    +	 * Config parameter indicating whether jobs can be uploaded and run from the web-frontend.
    +	 */
    +	public static final String JOB_MANAGER_WEB_SUBMISSION_KEY = "jobmanager.web.submit.allow";
    --- End diff --
    
    Why didn't you rename the config key?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155082996
  
    @rmetzger I did test, but on a slightly revision than this one. I'll check again.
    
    I did not realize usually jar sizes will exceed 10 MB. I must've underestimated. :')
    Will work on a fix to handle large uploads. I assume there are no issues with using netty example code, right? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45103976
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java ---
    @@ -0,0 +1,253 @@
    +/*
    + * 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 com.fasterxml.jackson.core.JsonGenerator;
    +import org.apache.flink.client.program.Client;
    +import org.apache.flink.client.program.PackagedProgram;
    +import org.apache.flink.client.program.ProgramInvocationException;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.optimizer.CompilerException;
    +import org.apache.flink.optimizer.DataStatistics;
    +import org.apache.flink.optimizer.Optimizer;
    +import org.apache.flink.optimizer.costs.DefaultCostEstimator;
    +import org.apache.flink.optimizer.plan.FlinkPlan;
    +import org.apache.flink.optimizer.plan.OptimizedPlan;
    +import org.apache.flink.optimizer.plan.StreamingPlan;
    +import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
    +import org.apache.flink.runtime.client.JobClient;
    +import org.apache.flink.runtime.client.JobExecutionException;
    +import org.apache.flink.runtime.instance.ActorGateway;
    +import org.apache.flink.runtime.jobgraph.JobGraph;
    +import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
    +import scala.concurrent.duration.FiniteDuration;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.io.PrintStream;
    +import java.io.PrintWriter;
    +import java.io.StringWriter;
    +import java.net.URISyntaxException;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * This handler handles requests to fetch plan for an already uploaded jar, as well as for
    + * running it.
    + */
    +public class JarActionHandler implements RequestHandler, RequestHandler.JsonResponse {
    +
    +	private final File jarDir;
    +
    +	private final boolean toRun;
    --- End diff --
    
    Two subclasses of an abstract class (one for Run, one for Plan) are nicer than a boolean flag that switches behavior.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155740233
  
    That's not a problem. You can test at your own convenience. There's no hurry. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-158765098
  
    @rmetzger 
    1. I have introduced a separate directory for uploads, named randomly, and deleted as part of the shut down hook. 
    2. I also tried closing the `URLClassLoader`, but it still doesn't allow for the Jars to be deleted. I will look further into it and see if I can get it to work. It shouldn't be a blocker however for merging this IMO. 
    
    @StephanEwen 
    1. I have removed the key to specify the upload directory, which takes care of naming of the config entry. It now is just `jobmanager.web.submit.allow`
    2. What would be the best way to separate the passing of path and query parameters? Of course, the most obvious choice is to change the method signature for `handleRequest` but it appears too much of a change, just to allow query params. Since the interface is designed by us, I think we can keep them together, and disallow any query param which overrides the path param. Otherwise, if we do indeed change this, I suggest a construct `Parameters` with fields `pathParams` and `queryParams`, with access method as `getParam(key, enum{PATH, QUERY})` This will be a much cleaner solution.
    3. I have some concerns about error handling. There are four handlers: 
      a. `StaticFileServerHandler`: Handles exception events itself, by sending an `INTERNAL_SERVER_ERROR`
      b. `RuntimeMonitorHandler`: Handles all exceptions itself, with either a `NOT_FOUND` or `INTERNAL_SERVER_ERROR` code.
      c. `HttpRequestHandler` [introduced in this PR]: Doesn't handler exceptions. But I'm inclined towards sending an `INTERNAL_SERVER_ERROR` code for any exceptions here.
      d. `PipelineErrorHandler` [introduced in this PR]: If an exception caught event is fired here, it can only happen because the netty threw an error [as reported by @gyfora in an instance]. There's really nothing to do here except sending an internal server error.
    What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156476686
  
    the jar has been build with scala 2.11 yes, flink with scala 2.10.
    But I still expect a good exception in the web frontend for this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156475956
  
    Hi @rmetzger , are you sure you built the source with scala 2.11? I was getting the same error you showed here, but building with scala 2.11 fixes it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45101180
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -635,8 +644,18 @@
     	 * The default number of archived jobs for the jobmanager
     	 */
     	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
    -	
    -	
    +
    +	/**
    +	 * By default, submitting jobs from the web-frontend is allowed.
    +	 */
    +	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMISSION = true;
    +
    +	/**
    +	 * Default directory for uploaded file storage for the Web frontend.
    +	 */
    +	public static final String DEFAULT_JOB_MANAGER_WEB_UPLOAD_DIR =
    +			(System.getProperty("java.io.tmpdir") == null ? "/tmp" : System.getProperty("java.io.tmpdir")) + "/webmonitor/";
    --- End diff --
    
    I thought this was considered a feature of the webclient that jars were persisted. I personally do not like it either. Should I do this then?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155738270
  
    Sure. :)
    It's probably best you didn't. There was still a minor thing to hash out. :')



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-163188173
  
    Hi @StephanEwen, I have modified the signature of the `handleRequest` method to separate path and query parameters.
    If there are any more concerns, let me know. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45091056
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/PipelineErrorHandler.java ---
    @@ -0,0 +1,79 @@
    +/*
    + * 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;
    +
    +import com.fasterxml.jackson.core.JsonGenerator;
    +import io.netty.buffer.Unpooled;
    +import io.netty.channel.ChannelHandler;
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.DefaultFullHttpResponse;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpResponseStatus;
    +import io.netty.handler.codec.http.HttpVersion;
    +import org.apache.flink.runtime.webmonitor.handlers.JsonFactory;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +
    +/**
    + * This is the last handler in the pipeline and logs all error messages.
    + */
    +@ChannelHandler.Sharable
    +public class PipelineErrorHandler extends SimpleChannelInboundHandler<Object> {
    +
    +	@Override
    +	protected void channelRead0(ChannelHandlerContext ctx, Object message) {
    +		// we can't deal with this message. No one in the pipeline handled it. Log it.
    +		System.err.println("Unknown message received: " + message);
    --- End diff --
    
    Ah. Forgot to change this. Will do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-157056790
  
    @gyfora this is the PR I was talking about regarding your *connection reset* exception. If you want, you can add the `PipelineErrorHandler` at the end of current pipeline and rerun your program to check the origin of that exception.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45202023
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +/*****************************************************************************
    + * This code is based on the "HttpUploadServerHandler" from the
    + * Netty project's HTTP server example.
    + *
    + * See http://netty.io and
    + * https://github.com/netty/netty/blob/netty-4.0.31.Final/example/src/main/java/io/netty/example/http/upload/HttpUploadServerHandler.java
    + *****************************************************************************/
    +
    +import io.netty.channel.ChannelHandlerContext;
    +import io.netty.channel.SimpleChannelInboundHandler;
    +import io.netty.handler.codec.http.HttpContent;
    +import io.netty.handler.codec.http.HttpHeaders;
    +import io.netty.handler.codec.http.HttpMethod;
    +import io.netty.handler.codec.http.HttpObject;
    +import io.netty.handler.codec.http.HttpRequest;
    +import io.netty.handler.codec.http.LastHttpContent;
    +import io.netty.handler.codec.http.QueryStringDecoder;
    +import io.netty.handler.codec.http.QueryStringEncoder;
    +import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory;
    +import io.netty.handler.codec.http.multipart.DiskFileUpload;
    +import io.netty.handler.codec.http.multipart.HttpDataFactory;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
    +import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder.EndOfDataDecoderException;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData;
    +import io.netty.handler.codec.http.multipart.InterfaceHttpData.HttpDataType;
    +
    +import java.io.File;
    +import java.util.UUID;
    +
    +/**
    + * Simple code which handles all HTTP requests from the user, and passes them to the Router
    + * handler directly if they do not involve file upload requests.
    + * If a file is required to be uploaded, it handles the upload, and in the http request to the
    + * next handler, passes the name of the file to the next handler.
    + */
    +public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject> {
    +
    +	private HttpRequest request;
    +
    +	private boolean readingChunks;
    +
    +	private static final HttpDataFactory factory = new DefaultHttpDataFactory(true); // use disk
    +
    +	private String requestPath;
    +
    +	private HttpPostRequestDecoder decoder;
    +
    +	private final File uploadDir;
    +
    +	/**
    +	 * The directory where files should be uploaded.
    +	 */
    +	public HttpRequestHandler(File uploadDir) {
    +		this.uploadDir = uploadDir;
    +	}
    +
    +	@Override
    +	public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
    +		if (decoder != null) {
    +			decoder.cleanFiles();
    +		}
    +	}
    +
    +	@Override
    +	public void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Exception {
    +		if (msg instanceof HttpRequest) {
    +			request = (HttpRequest) msg;
    +			requestPath = new QueryStringDecoder(request.getUri()).path();
    +			if (request.getMethod() != HttpMethod.POST) {
    --- End diff --
    
    I am just curious why POST needs a special path and the other HTTP action do not...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-156450181
  
    here is the jar file: https://transfer.sh/KNAnm/flink0.10-scala2.11-1.0-snapshot.jar
    
    it has been build from here: https://github.com/rmetzger/scratch/tree/flink0.10-scala2.11/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45100162
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -635,8 +644,18 @@
     	 * The default number of archived jobs for the jobmanager
     	 */
     	public static final int DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT = 5;
    -	
    -	
    +
    +	/**
    +	 * By default, submitting jobs from the web-frontend is allowed.
    +	 */
    +	public static final boolean DEFAULT_JOB_MANAGER_WEB_SUBMISSION = true;
    +
    +	/**
    +	 * Default directory for uploaded file storage for the Web frontend.
    +	 */
    +	public static final String DEFAULT_JOB_MANAGER_WEB_UPLOAD_DIR =
    +			(System.getProperty("java.io.tmpdir") == null ? "/tmp" : System.getProperty("java.io.tmpdir")) + "/webmonitor/";
    --- End diff --
    
    How about handling the temp directory similar to the IOManager and BlobManager: We create directory in the temporary directory (with a random name) and use a shutdown hook to remove the directory on shutdown again.
    
    I don't like keeping the jar files because it will pollute the directory on the server.
    Also the current approach does not allow running multiple JobManagers at the same machine (something that can easily happen with YARN).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r46759219
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java ---
    @@ -302,6 +302,15 @@
     	 */
     	public static final String JOB_MANAGER_WEB_LOG_PATH_KEY = "jobmanager.web.log.path";
     
    +	/**
    +	 * Config parameter indicating whether jobs can be uploaded and run from the web-frontend.
    +	 */
    +	public static final String JOB_MANAGER_WEB_SUBMISSION_KEY = "jobmanager.web.submit.allow";
    --- End diff --
    
    Sorry about the delay.
    What should I rename it to? :confused: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-165754941
  
    Olrite. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1338#discussion_r45104323
  
    --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java ---
    @@ -113,7 +117,17 @@ private void respondAsLeader(ChannelHandlerContext ctx, Routed routed, ActorGate
     		DefaultFullHttpResponse response;
     
     		try {
    -			String result = handler.handleRequest(routed.pathParams(), jobManager);
    +			Map<String, String> params = routed.pathParams();
    --- End diff --
    
    I guess separation would be the better solution.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-165744791
  
    @StephanEwen can you have a look at this again? Apologies for being hasty, but I've already addressed all concerns; this should be mergeable now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155518122
  
    @rmetzger Good to test. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-155102201
  
    Good idea.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1338#issuecomment-162340095
  
    @rmetzger thanks. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2978][web-dashboard][webclient] Integra...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/1338


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---