You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by mx...@apache.org on 2015/07/22 14:43:15 UTC

flink git commit: [FLINK-2218] Web client cannot distinguesh between Flink options and program arguments

Repository: flink
Updated Branches:
  refs/heads/master 13762149d -> 72b5dc980


[FLINK-2218] Web client cannot distinguesh between Flink options and program arguments

- added new input fields 'options' to WebClient
- adapted WebClient-to-JobManager job submission logic
- removed reduncand code and re-used CliFrontend

- updated documentation (including new screenshots)
- (some additional minor cleanup in launch.html and program.js)

This closes #904.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/72b5dc98
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/72b5dc98
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/72b5dc98

Branch: refs/heads/master
Commit: 72b5dc980b64fee10c886c37adcd0e878f889d91
Parents: 1376214
Author: mjsax <mj...@informatik.hu-berlin.de>
Authored: Fri Jul 10 16:53:13 2015 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Wed Jul 22 14:39:36 2015 +0200

----------------------------------------------------------------------
 docs/apis/web_client.md                         |  13 +-
 docs/page/img/jobmanager.png                    | Bin 0 -> 97051 bytes
 .../compiler-webclient-new.png                  | Bin 134963 -> 0 bytes
 .../jobmanager-running-new.png                  | Bin 132859 -> 0 bytes
 .../img/quickstart-example/run-webclient.png    | Bin 89469 -> 0 bytes
 docs/page/img/webclient_job_view.png            | Bin 0 -> 64972 bytes
 docs/page/img/webclient_plan_view.png           | Bin 0 -> 140756 bytes
 docs/quickstart/run_example_quickstart.md       |  16 +-
 .../org/apache/flink/client/CliFrontend.java    | 131 +++++++----
 .../org/apache/flink/client/WebFrontend.java    |   3 +-
 .../flink/client/cli/CliFrontendParser.java     |   2 +-
 .../flink/client/web/JobSubmissionServlet.java  | 222 ++++++++-----------
 .../flink/client/web/WebInterfaceServer.java    |  12 +-
 .../src/main/resources/web-docs/js/program.js   |  21 +-
 .../src/main/resources/web-docs/launch.html     | 219 +++++++++---------
 15 files changed, 323 insertions(+), 316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/docs/apis/web_client.md
----------------------------------------------------------------------
diff --git a/docs/apis/web_client.md b/docs/apis/web_client.md
index d3680f1..8e3c84a 100644
--- a/docs/apis/web_client.md
+++ b/docs/apis/web_client.md
@@ -51,15 +51,22 @@ The interface starts serving the job view.
 You can **upload** a Flink program as a jar file. To **execute** an uploaded program:
 
 * select it from the job list on the left, 
-* enter the program arguments in the *"Arguments"* field (bottom left), and 
+* enter (optional) execution options in the *"Flink Options"* field (bottom left),
+* enter (optional) program arguments in the *"Program Arguments"* field (bottom left), and
 * click on the *"Run Job"* button (bottom right).
 
 If the *“Show optimizer plan”* option is enabled (default), the *plan view* is display next, otherwise the job is directly submitted to the JobManager for execution.
 
-The web interface can also handle multiple Flink jobs within a single jar file. To use this feature, package all required class files of all jobs into a single jar and specify the entry classes for each job as comma-separated-values in *program-class* argument within the jar's manifest file. The job view displays each entry class and you can pick any of them to preview the plan and/or submit the job to the JobManager. In case the jar's manifest file does not specify any entry class, you can specify it before the argument list as:
+The web interface can also handle multiple Flink jobs within a single jar file. To use this feature, package all required class files of all jobs into a single jar and specify the entry classes for each job as comma-separated-values in *program-class* argument within the jar's manifest file. The job view displays each entry class and you can pick any of them to preview the plan and/or submit the job to the JobManager. In case the jar's manifest file does not specify any entry class, you can specify it in the options field as:
 
 ```
--c <assemblerClass> <programArgs...>
+-c <assemblerClass>
+```
+
+It is also possible to set the default parallelism for the execution in options field as:
+
+```
+-p <dop>
 ```
 
 Furthermore, for each entry class implementing ```ProgramDescription``` interface, the provided description is shown as tooltip for the job (see {% gh_link flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCountMeta.java  "WordCountMeta example" %}).

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/docs/page/img/jobmanager.png
----------------------------------------------------------------------
diff --git a/docs/page/img/jobmanager.png b/docs/page/img/jobmanager.png
new file mode 100644
index 0000000..eaa5838
Binary files /dev/null and b/docs/page/img/jobmanager.png differ

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/docs/page/img/quickstart-example/compiler-webclient-new.png
----------------------------------------------------------------------
diff --git a/docs/page/img/quickstart-example/compiler-webclient-new.png b/docs/page/img/quickstart-example/compiler-webclient-new.png
deleted file mode 100644
index e60689e..0000000
Binary files a/docs/page/img/quickstart-example/compiler-webclient-new.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/docs/page/img/quickstart-example/jobmanager-running-new.png
----------------------------------------------------------------------
diff --git a/docs/page/img/quickstart-example/jobmanager-running-new.png b/docs/page/img/quickstart-example/jobmanager-running-new.png
deleted file mode 100644
index 6255022..0000000
Binary files a/docs/page/img/quickstart-example/jobmanager-running-new.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/docs/page/img/quickstart-example/run-webclient.png
----------------------------------------------------------------------
diff --git a/docs/page/img/quickstart-example/run-webclient.png b/docs/page/img/quickstart-example/run-webclient.png
deleted file mode 100644
index 8b3d535..0000000
Binary files a/docs/page/img/quickstart-example/run-webclient.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/docs/page/img/webclient_job_view.png
----------------------------------------------------------------------
diff --git a/docs/page/img/webclient_job_view.png b/docs/page/img/webclient_job_view.png
new file mode 100644
index 0000000..dbd64c7
Binary files /dev/null and b/docs/page/img/webclient_job_view.png differ

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/docs/page/img/webclient_plan_view.png
----------------------------------------------------------------------
diff --git a/docs/page/img/webclient_plan_view.png b/docs/page/img/webclient_plan_view.png
new file mode 100644
index 0000000..201ef66
Binary files /dev/null and b/docs/page/img/webclient_plan_view.png differ

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/docs/quickstart/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/run_example_quickstart.md b/docs/quickstart/run_example_quickstart.md
index b7f21c4..fb1d798 100644
--- a/docs/quickstart/run_example_quickstart.md
+++ b/docs/quickstart/run_example_quickstart.md
@@ -87,7 +87,7 @@ The Flink web client allows to submit Flink programs using a graphical user inte
 
 <div class="row" style="padding-top:15px">
 	<div class="col-md-6">
-		<a data-lightbox="compiler" href="{{ site.baseurl }}/page/img/quickstart-example/run-webclient.png" data-lightbox="example-1"><img class="img-responsive" src="{{ site.baseurl }}/page/img/quickstart-example/run-webclient.png" /></a>
+		<a data-lightbox="compiler" href="{{ site.baseurl }}/page/img/webclient_job_view.png" data-lightbox="example-1"><img class="img-responsive" src="{{ site.baseurl }}/page/img/webclient_job_view.png" /></a>
 	</div>
 	<div class="col-md-6">
 		1. Open web client on  <a href="http://localhost:8080/launch.html">localhost:8080</a> <br>
@@ -96,7 +96,8 @@ The Flink web client allows to submit Flink programs using a graphical user inte
 			./examples/flink-java-examples-*-KMeans.jar
 			{% endhighlight %} </br>
 		3. Select it in the left box to see how the operators in the plan are connected to each other. <br>
-		4. Enter the arguments in the lower left box:
+		4. Enter the arguments and options in the lower left box: <br>
+            Arguments: <br>
 			{% highlight bash %}
 			file://<pathToFlink>/kmeans/points file://<pathToFlink>/kmeans/centers file://<pathToFlink>/kmeans/result 10
 			{% endhighlight %}
@@ -104,12 +105,16 @@ The Flink web client allows to submit Flink programs using a graphical user inte
 			{% highlight bash %}
 			file:///tmp/flink/kmeans/points file:///tmp/flink/kmeans/centers file:///tmp/flink/kmeans/result 10
 			{% endhighlight %}
+            Options (optional): (set the default parallelims, e.g., to 4) <br>
+			{% highlight bash %}
+            -p 4
+			{% endhighlight %}
 	</div>
 </div>
 <hr>
 <div class="row" style="padding-top:15px">
 	<div class="col-md-6">
-		<a data-lightbox="compiler" href="{{ site.baseurl }}/page/img/quickstart-example/compiler-webclient-new.png" data-lightbox="example-1"><img class="img-responsive" src="{{ site.baseurl }}/page/img/quickstart-example/compiler-webclient-new.png" /></a>
+		<a data-lightbox="compiler" href="{{ site.baseurl }}/page/img/webclient_plan_view.png" data-lightbox="example-1"><img class="img-responsive" src="{{ site.baseurl }}/page/img/webclient_plan_view.png" /></a>
 	</div>
 
 	<div class="col-md-6">
@@ -120,7 +125,7 @@ The Flink web client allows to submit Flink programs using a graphical user inte
 <hr>
 <div class="row" style="padding-top:15px">
 	<div class="col-md-6">
-		<a data-lightbox="compiler" href="{{ site.baseurl }}/page/img/quickstart-example/jobmanager-running-new.png" data-lightbox="example-1"><img class="img-responsive" src="{{ site.baseurl }}/page/img/quickstart-example/jobmanager-running-new.png" /></a>
+		<a data-lightbox="compiler" href="{{ site.baseurl }}/page/img/jobmanager.png" data-lightbox="example-1"><img class="img-responsive" src="{{ site.baseurl }}/page/img/jobmanager.png" /></a>
 	</div>
 	<div class="col-md-6">
 		1. Press the <b>Continue</b> button to start executing the job. <br>
@@ -152,4 +157,5 @@ The following three pictures show the results for the sample input above. Play a
 
 |relative stddev = 0.03|relative stddev = 0.08|relative stddev = 0.15|
 |:--------------------:|:--------------------:|:--------------------:|
-|<img src="{{ site.baseurl }}/page/img/quickstart-example/result003.png" alt="example1" style="width: 275px;"/>|<img src="{{ site.baseurl }}/page/img/quickstart-example/result008.png" alt="example2" style="width: 275px;"/>|<img src="{{ site.baseurl }}/page/img/quickstart-example/result015.png" alt="example3" style="width: 275px;"/>|
\ No newline at end of file
+|<img src="{{ site.baseurl }}/page/img/quickstart-example/result003.png" alt="example1" style="width: 275px;"/>|<img src="{{ site.baseurl }}/page/img/quickstart-example/result008.png" alt="example2" style="width: 275px;"/>|<img src="{{ site.baseurl }}/page/img/quickstart-example/result015.png" alt="example3" style="width: 275px;"/>|
+

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index 83bb99a..e28f2fd 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -46,7 +46,6 @@ import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.client.cli.CancelOptions;
 import org.apache.flink.client.cli.CliArgsException;
 import org.apache.flink.client.cli.CliFrontendParser;
-
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -62,12 +61,16 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.optimizer.plan.FlinkPlan;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.runtime.client.JobStatusMessage;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.security.SecurityUtils;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
@@ -90,8 +93,8 @@ import scala.concurrent.duration.FiniteDuration;
 public class CliFrontend {
 
 	// actions
-	private static final String ACTION_RUN = "run";
-	private static final String ACTION_INFO = "info";
+	public static final String ACTION_RUN = "run";
+	public static final String ACTION_INFO = "info";
 	private static final String ACTION_LIST = "list";
 	private static final String ACTION_CANCEL = "cancel";
 
@@ -99,7 +102,7 @@ public class CliFrontend {
 	private static final String ENV_CONFIG_DIRECTORY = "FLINK_CONF_DIR";
 	private static final String CONFIG_DIRECTORY_FALLBACK_1 = "../conf";
 	private static final String CONFIG_DIRECTORY_FALLBACK_2 = "conf";
-	
+
 	// YARN-session related constants
 	public static final String YARN_PROPERTIES_FILE = ".yarn-properties";
 	public static final String YARN_PROPERTIES_JOBMANAGER_KEY = "jobManager";
@@ -111,9 +114,8 @@ public class CliFrontend {
 	/**
 	 * A special host name used to run a job by deploying Flink into a YARN cluster,
 	 * if this string is specified as the JobManager address
- 	 */
+	 */
 	public static final String YARN_DEPLOY_JOBMANAGER = "yarn-cluster";
-	
 
 	// --------------------------------------------------------------------------------------------
 	// --------------------------------------------------------------------------------------------
@@ -134,9 +136,15 @@ public class CliFrontend {
 
 	private AbstractFlinkYarnCluster yarnCluster;
 
+	static boolean webFrontend = false;
+
+	private FlinkPlan optimizedPlan;
+
+	private JobGraph jobGraph;
+
 	/**
 	 *
-	 * @throws Exception Thrown if teh configuration directory was not found, the configuration could not
+	 * @throws Exception Thrown if the configuration directory was not found, the configuration could not
 	 *                   be loaded, or the YARN properties could not be parsed.
 	 */
 	public CliFrontend() throws Exception {
@@ -214,11 +222,11 @@ public class CliFrontend {
 		this.lookupTimeout = AkkaUtils.getLookupTimeout(config);
 	}
 
-	
+
 	// --------------------------------------------------------------------------------------------
 	//  Execute Actions
 	// --------------------------------------------------------------------------------------------
-	
+
 	/**
 	 * Executions the run action.
 	 * 
@@ -351,7 +359,7 @@ public class CliFrontend {
 		}
 
 		// -------- build the packaged program -------------
-		
+
 		PackagedProgram program;
 		try {
 			LOG.info("Building program from JAR file");
@@ -360,31 +368,39 @@ public class CliFrontend {
 		catch (Throwable t) {
 			return handleError(t);
 		}
-		
+
 		try {
 			int parallelism = options.getParallelism();
 
 			LOG.info("Creating program plan dump");
 			Client client = getClient(options, program.getUserCodeClassLoader(), program.getMainClassName(), parallelism);
-			String jsonPlan = client.getOptimizedPlanAsJson(program, parallelism);
+			FlinkPlan flinkPlan = client.getOptimizedPlan(program, parallelism);
 
-			if (jsonPlan != null) {
-				System.out.println("----------------------- Execution Plan -----------------------");
-				System.out.println(jsonPlan);
-				System.out.println("--------------------------------------------------------------");
-			}
-			else {
-				System.out.println("JSON plan could not be generated.");
-			}
+			if (webFrontend) {
+				this.optimizedPlan = flinkPlan;
+				this.jobGraph = client.getJobGraph(program, flinkPlan);
+			} else {
+				String jsonPlan = new PlanJSONDumpGenerator()
+						.getOptimizerPlanAsJSON((OptimizedPlan) flinkPlan);
 
-			String description = program.getDescription();
-			if (description != null) {
-				System.out.println();
-				System.out.println(description);
-			}
-			else {
-				System.out.println();
-				System.out.println("No description provided.");
+				if (jsonPlan != null) {
+					System.out.println("----------------------- Execution Plan -----------------------");
+					System.out.println(jsonPlan);
+					System.out.println("--------------------------------------------------------------");
+				}
+				else {
+					System.out.println("JSON plan could not be generated.");
+				}
+
+				String description = program.getDescription();
+				if (description != null) {
+					System.out.println();
+					System.out.println(description);
+				}
+				else {
+					System.out.println();
+					System.out.println("No description provided.");
+				}
 			}
 			return 0;
 		}
@@ -423,13 +439,13 @@ public class CliFrontend {
 
 		boolean running = options.getRunning();
 		boolean scheduled = options.getScheduled();
-		
+
 		// print running and scheduled jobs if not option supplied
 		if (!running && !scheduled) {
 			running = true;
 			scheduled = true;
 		}
-		
+
 		try {
 			ActorRef jobManager = getJobManager(options);
 
@@ -517,7 +533,7 @@ public class CliFrontend {
 			return handleError(t);
 		}
 	}
-	
+
 	/**
 	 * Executes the CANCEL action.
 	 * 
@@ -542,7 +558,7 @@ public class CliFrontend {
 			CliFrontendParser.printHelpForCancel();
 			return 0;
 		}
-		
+
 		String[] cleanedArgs = options.getArgs();
 		JobID jobId;
 
@@ -562,7 +578,7 @@ public class CliFrontend {
 			System.out.println("Error: Specify a Job ID to cancel a job.");
 			return 1;
 		}
-		
+
 		try {
 			ActorRef jobManager = getJobManager(options);
 			Future<Object> response = Patterns.ask(jobManager, new CancelJob(jobId), new Timeout(askTimeout));
@@ -612,14 +628,18 @@ public class CliFrontend {
 				}
 				yarnCluster.stopAfterJob(execResult.getJobID());
 				yarnCluster.disconnect();
-				System.out.println("The Job has been submitted with JobID "+execResult.getJobID());
+				if(!webFrontend) {
+					System.out.println("The Job has been submitted with JobID "+execResult.getJobID());
+				}
 				return 0;
 			}
 			if (execResult instanceof JobExecutionResult) {
 				JobExecutionResult result = (JobExecutionResult) execResult;
-				System.out.println("Job Runtime: " + result.getNetRuntime() + " ms");
+				if(!webFrontend) {
+					System.out.println("Job Runtime: " + result.getNetRuntime() + " ms");
+				}
 				Map<String, Object> accumulatorsResult = result.getAllAccumulatorResults();
-				if (accumulatorsResult.size() > 0) {
+				if (accumulatorsResult.size() > 0 && !webFrontend) {
 					System.out.println("Accumulator Results: ");
 					System.out.println(AccumulatorHelper.getResultsFormated(accumulatorsResult));
 				}
@@ -647,7 +667,7 @@ public class CliFrontend {
 		}
 
 		File jarFile = new File(jarFilePath);
-		
+
 		// Check if JAR file exists
 		if (!jarFile.exists()) {
 			throw new FileNotFoundException("JAR file does not exist: " + jarFile);
@@ -655,7 +675,7 @@ public class CliFrontend {
 		else if (!jarFile.isFile()) {
 			throw new FileNotFoundException("JAR file is not a file: " + jarFile);
 		}
-		
+
 		// Get assembler class
 		String entryPointClass = options.getEntryPointClassName();
 
@@ -664,7 +684,6 @@ public class CliFrontend {
 				new PackagedProgram(jarFile, entryPointClass, programArgs);
 	}
 
-
 	protected InetSocketAddress getJobManagerAddress(CommandLineOptions options) throws Exception {
 
 		// first, check if the address is specified as an option
@@ -702,7 +721,7 @@ public class CliFrontend {
 
 		return jobManagerAddress;
 	}
-	
+
 	protected ActorRef getJobManager(CommandLineOptions options) throws Exception {
 		//TODO: Get ActorRef from YarnCluster if we are in YARN mode.
 
@@ -729,7 +748,6 @@ public class CliFrontend {
 		return jmActor;
 	}
 
-
 	/**
 	 * @param userParallelism The parallelism requested by the user in the CLI frontend.
 	 */
@@ -742,10 +760,10 @@ public class CliFrontend {
 			// user wants to run Flink in YARN cluster.
 			CommandLine commandLine = options.getCommandLine();
 			AbstractFlinkYarnClient flinkYarnClient = CliFrontendParser.getFlinkYarnSessionCli().createFlinkYarnClient(commandLine);
-			flinkYarnClient.setName("Flink Application: " + programName);
 			if (flinkYarnClient == null) {
 				throw new RuntimeException("Unable to create Flink YARN Client. Check previous log messages");
 			}
+			flinkYarnClient.setName("Flink Application: " + programName);
 
 			// the number of slots available from YARN:
 			int yarnTmSlots = flinkYarnClient.getTaskManagerSlots();
@@ -814,6 +832,9 @@ public class CliFrontend {
 	 * @return The return code for the process.
 	 */
 	private int handleArgException(Exception e) {
+		if (webFrontend) {
+			throw new RuntimeException(e);
+		}
 		LOG.error("Invalid command line arguments." + (e.getMessage() == null ? "" : e.getMessage()));
 
 		System.out.println(e.getMessage());
@@ -821,6 +842,7 @@ public class CliFrontend {
 		System.out.println("Use the help option (-h or --help) to get help on the command.");
 		return 1;
 	}
+
 	/**
 	 * Displays an exception message.
 	 * 
@@ -828,6 +850,9 @@ public class CliFrontend {
 	 * @return The return code for the process.
 	 */
 	private int handleError(Throwable t) {
+		if (webFrontend) {
+			throw new RuntimeException(t);
+		}
 		LOG.error("Error while running the command.", t);
 
 		t.printStackTrace();
@@ -838,7 +863,9 @@ public class CliFrontend {
 
 	private void logAndSysout(String message) {
 		LOG.info(message);
-		System.out.println(message);
+		if (!webFrontend) {
+			System.out.println(message);
+		}
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -852,27 +879,29 @@ public class CliFrontend {
 	 * @return The return code of the program
 	 */
 	public int parseParameters(String[] args) {
-		
+
 		// check for action
 		if (args.length < 1) {
 			CliFrontendParser.printHelp();
 			System.out.println("Please specify an action.");
 			return 1;
 		}
-		
+
 		// get action
 		String action = args[0];
 
 		// remove action from parameters
 		final String[] params = Arrays.copyOfRange(args, 1, args.length);
-		
+
 		// do action
 		if (action.equals(ACTION_RUN)) {
 			// run() needs to run in a secured environment for the optimizer.
 			if (SecurityUtils.isSecurityEnabled()) {
 				String message = "Secure Hadoop environment setup detected. Running in secure context.";
 				LOG.info(message);
-				System.out.println(message);
+				if (!webFrontend) {
+					System.out.println(message);
+				}
 
 				try {
 					return SecurityUtils.runSecured(new SecurityUtils.FlinkSecuredRunner<Integer>() {
@@ -910,6 +939,14 @@ public class CliFrontend {
 		}
 	}
 
+	public FlinkPlan getFlinkPlan() {
+		return this.optimizedPlan;
+	}
+
+	public JobGraph getJobGraph() {
+		return this.jobGraph;
+	}
+
 	public void shutdown() {
 		ActorSystem sys = this.actorSystem;
 		if (sys != null) {

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/flink-clients/src/main/java/org/apache/flink/client/WebFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/WebFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/WebFrontend.java
index 9587ab2..39c8405 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/WebFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/WebFrontend.java
@@ -66,7 +66,8 @@ public class WebFrontend {
 										ConfigConstants.DEFAULT_WEBCLIENT_PORT);
 
 			// start the server
-			WebInterfaceServer server = new WebInterfaceServer(config, port);
+			CliFrontend.webFrontend = true;
+			WebInterfaceServer server = new WebInterfaceServer(args[1], config, port);
 			LOG.info("Starting web frontend server on port " + port + '.');
 			server.start();
 			server.join();

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
index 0f6ad24..2e66a97 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java
@@ -43,7 +43,7 @@ public class CliFrontendParser {
 
 	static final Option JAR_OPTION = new Option("j", "jarfile", true, "Flink program JAR file.");
 
-	static final Option CLASS_OPTION = new Option("c", "class", true,
+	public static final Option CLASS_OPTION = new Option("c", "class", true,
 			"Class with the program entry point (\"main\" method or \"getPlan()\" method. Only needed if the " +
 					"JAR file does not specify the class in its manifest.");
 

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/flink-clients/src/main/java/org/apache/flink/client/web/JobSubmissionServlet.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/web/JobSubmissionServlet.java b/flink-clients/src/main/java/org/apache/flink/client/web/JobSubmissionServlet.java
index 5ec698b..fed3546 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/web/JobSubmissionServlet.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/web/JobSubmissionServlet.java
@@ -36,15 +36,16 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.flink.client.CliFrontend;
+import org.apache.flink.client.cli.CliFrontendParser;
 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.optimizer.CompilerException;
 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.plandump.PlanJSONDumpGenerator;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -69,6 +70,8 @@ public class JobSubmissionServlet extends HttpServlet {
 
 	private static final String ACTION_BACK_VALUE = "back";
 
+	private static final String OPTIONS_PARAM_NAME = "options";
+
 	private static final String JOB_PARAM_NAME = "job";
 
 	private static final String CLASS_PARAM_NAME = "assemblerClass";
@@ -91,11 +94,12 @@ public class JobSubmissionServlet extends HttpServlet {
 
 	private final Random rand;							// random number generator for UID
 
-	private final Configuration config;
+	private final CliFrontend cli;
+
 
 
-	public JobSubmissionServlet(Configuration config, File jobDir, File planDir) {
-		this.config = config;
+	public JobSubmissionServlet(CliFrontend cli, File jobDir, File planDir) {
+		this.cli = cli;
 		this.jobStoreDirectory = jobDir;
 		this.planDumpDirectory = planDir;
 
@@ -116,16 +120,17 @@ public class JobSubmissionServlet extends HttpServlet {
 			// --------------- submit a job -------------------
 
 			// get the parameters
+			String options = req.getParameter(OPTIONS_PARAM_NAME);
 			String jobName = req.getParameter(JOB_PARAM_NAME);
 			String assemblerClass = req.getParameter(CLASS_PARAM_NAME);
-			String args = req.getParameter(ARGUMENTS_PARAM_NAME);
+			String arguments = req.getParameter(ARGUMENTS_PARAM_NAME);
 			String showPlan = req.getParameter(SHOW_PLAN_PARAM_NAME);
 			String suspendPlan = req.getParameter(SUSPEND_PARAM_NAME);
 
-			// check that all parameters are set
-			// do NOT check 'assemblerClass' -> it is OK if it is not set
+			// check that parameters are set
+			// do NOT check 'options' or 'assemblerClass' -> it is OK if not set
 			if (checkParameterSet(resp, jobName, JOB_PARAM_NAME)
-				|| checkParameterSet(resp, args, ARGUMENTS_PARAM_NAME)
+				|| checkParameterSet(resp, arguments, ARGUMENTS_PARAM_NAME)
 				|| checkParameterSet(resp, showPlan, SHOW_PLAN_PARAM_NAME)
 				|| checkParameterSet(resp, suspendPlan, SUSPEND_PARAM_NAME))
 			{
@@ -135,106 +140,92 @@ public class JobSubmissionServlet extends HttpServlet {
 			boolean show = Boolean.parseBoolean(showPlan);
 			boolean suspend = Boolean.parseBoolean(suspendPlan);
 
-			// check, if the jar exists
-			File jarFile = new File(jobStoreDirectory, jobName);
-			if (!jarFile.exists()) {
-				showErrorPage(resp, "The jar file + '" + jarFile.getPath() + "' does not exist.");
+			List<String> cliOptions;
+			try {
+				cliOptions = tokenizeArguments(options);
+			} catch (IllegalArgumentException iaex) {
+				showErrorPage(resp, "Flink options contain an unterminated quoted string.");
 				return;
 			}
 
-			// parse the arguments
-			List<String> params;
+			List<String> cliArguments;
 			try {
-				params = tokenizeArguments(args);
+				cliArguments = tokenizeArguments(arguments);
 			} catch (IllegalArgumentException iaex) {
-				showErrorPage(resp, "The arguments contain an unterminated quoted string.");
+				showErrorPage(resp, "Program arguments contain an unterminated quoted string.");
 				return;
 			}
 
-			int parallelism = -1;
-			while(params.size() >= 2) {
-				if (params.get(0).equals("-c")) {
-					assemblerClass = params.get(1);
-					params.remove(0);
-					params.remove(0);
-				}
-				else if (params.get(0).equals("-p")) {
-					parallelism = Integer.parseInt(params.get(1));
-					params.remove(0);
-					params.remove(0);
-				}
-				else {
-					break;
-				}
+			String[] args = new String[1 + (assemblerClass == null ? 0 : 2) + cliOptions.size() + 1 + cliArguments.size()];
+
+			List<String> parameters = new ArrayList<String>(args.length);
+			parameters.add(CliFrontend.ACTION_INFO);
+			parameters.addAll(cliOptions);
+			if (assemblerClass != null) {
+				parameters.add("-" + CliFrontendParser.CLASS_OPTION.getOpt());
+				parameters.add(assemblerClass);
 			}
+			parameters.add(jobStoreDirectory + File.separator + jobName);
+			parameters.addAll(cliArguments);
 
-			// create the plan
-			String[] options = params.isEmpty() ? new String[0] : params.toArray(new String[params.size()]);
-			PackagedProgram program;
 			FlinkPlan optPlan;
-			Client client;
-
 			try {
-				if (assemblerClass == null) {
-					program = new PackagedProgram(jarFile, options);
-				} else {
-					program = new PackagedProgram(jarFile, assemblerClass, options);
-				}
-
-				client = new Client(config, program.getUserCodeClassLoader());
-
-				optPlan = client.getOptimizedPlan(program, parallelism);
+				this.cli.parseParameters(parameters.toArray(args));
 
+				optPlan = this.cli.getFlinkPlan();
 				if (optPlan == null) {
-					throw new Exception("The optimized plan could not be produced.");
+					// wrapping hack to get this exception handled correctly by following catch block
+					throw new RuntimeException(new Exception("The optimized plan could not be produced."));
 				}
 			}
-			catch (ProgramInvocationException e) {
-				// collect the stack trace
-				StringWriter sw = new StringWriter();
-				PrintWriter w = new PrintWriter(sw);
-
-				if (e.getCause() == null) {
-					e.printStackTrace(w);
-				} else {
-					e.getCause().printStackTrace(w);
-				}
+			catch (RuntimeException e) {
+				Throwable t = e.getCause();
+
+				if(t instanceof ProgramInvocationException) {
+					// collect the stack trace
+					StringWriter sw = new StringWriter();
+					PrintWriter w = new PrintWriter(sw);
+
+					if (t.getCause() == null) {
+						t.printStackTrace(w);
+					} else {
+						t.getCause().printStackTrace(w);
+					}
 
-				String message = sw.toString();
-				message = StringEscapeUtils.escapeHtml4(message);
+					String message = sw.toString();
+					message = StringEscapeUtils.escapeHtml4(message);
 
-				showErrorPage(resp, "An error occurred while invoking the program:<br/><br/>"
-					+ e.getMessage() + "<br/>"
-					+ "<br/><br/><pre>" + message + "</pre>");
-				return;
-			}
-			catch (CompilerException cex) {
-				// collect the stack trace
-				StringWriter sw = new StringWriter();
-				PrintWriter w = new PrintWriter(sw);
-				cex.printStackTrace(w);
-
-				String message = sw.toString();
-				message = StringEscapeUtils.escapeHtml4(message);
-
-				showErrorPage(resp, "An error occurred in the compiler:<br/><br/>"
-					+ cex.getMessage() + "<br/>"
-					+ (cex.getCause() != null ? "Caused by: " + cex.getCause().getMessage():"")
-					+ "<br/><br/><pre>" + message + "</pre>");
-				return;
-			}
-			catch (Throwable t) {
-				// collect the stack trace
-				StringWriter sw = new StringWriter();
-				PrintWriter w = new PrintWriter(sw);
-				t.printStackTrace(w);
+					showErrorPage(resp, "An error occurred while invoking the program:<br/><br/>"
+							+ t.getMessage() + "<br/>"
+							+ "<br/><br/><pre>" + message + "</pre>");
+					return;
+				} else if (t instanceof CompilerException) {
+					// collect the stack trace
+					StringWriter sw = new StringWriter();
+					PrintWriter w = new PrintWriter(sw);
+					t.printStackTrace(w);
+
+					String message = sw.toString();
+					message = StringEscapeUtils.escapeHtml4(message);
+
+					showErrorPage(resp, "An error occurred in the compiler:<br/><br/>"
+							+ t.getMessage() + "<br/>"
+							+ (t.getCause() != null ? "Caused by: " + t.getCause().getMessage():"")
+							+ "<br/><br/><pre>" + message + "</pre>");
+					return;
+				} else {
+					// collect the stack trace
+					StringWriter sw = new StringWriter();
+					PrintWriter w = new PrintWriter(sw);
+					t.printStackTrace(w);
 
-				String message = sw.toString();
-				message = StringEscapeUtils.escapeHtml4(message);
+					String message = sw.toString();
+					message = StringEscapeUtils.escapeHtml4(message);
 
-				showErrorPage(resp, "An unexpected error occurred:<br/><br/>" + t.getMessage() + "<br/><br/><pre>"
-					+ message + "</pre>");
-				return;
+					showErrorPage(resp, "An unexpected error occurred:<br/><br/>" + t.getMessage() + "<br/><br/><pre>"
+							+ message + "</pre>");
+					return;
+				}
 			}
 
 			// redirect according to our options
@@ -262,38 +253,18 @@ public class JobSubmissionServlet extends HttpServlet {
 
 				// submit the job only, if it should not be suspended
 				if (!suspend) {
-					if (optPlan instanceof OptimizedPlan) {
-						try {
-							client.run(program, (OptimizedPlan) optPlan, false);
-						}
-						catch (Throwable t) {
-							LOG.error("Error submitting job to the job-manager.", t);
-							showErrorPage(resp, t.getMessage());
-							return;
-						}
-						finally {
-							program.deleteExtractedLibraries();
-						}
-					}
-					else {
-						throw new RuntimeException("Not implemented for Streaming Job plans");
-					}
-				}
-				else {
+					parameters.set(0, CliFrontend.ACTION_RUN);
 					try {
-						this.submittedJobs.put(uid, client.getJobGraph(program, optPlan));
-					}
-					catch (ProgramInvocationException piex) {
-						LOG.error("Error creating JobGraph from optimized plan.", piex);
-						showErrorPage(resp, piex.getMessage());
-						return;
-					}
-					catch (Throwable t) {
-						LOG.error("Error creating JobGraph from optimized plan.", t);
-						showErrorPage(resp, t.getMessage());
+						this.cli.parseParameters(parameters.toArray(args));
+					} catch(RuntimeException e) {
+						LOG.error("Error submitting job to the job-manager.", e.getCause());
+						showErrorPage(resp, e.getCause().getMessage());
 						return;
 					}
 				}
+				else {
+					this.submittedJobs.put(uid, this.cli.getJobGraph());
+				}
 
 				// redirect to the plan display page
 				resp.sendRedirect("showPlan?id=" + uid + "&suspended=" + (suspend ? "true" : "false"));
@@ -301,19 +272,17 @@ public class JobSubmissionServlet extends HttpServlet {
 			else {
 				// don't show any plan. directly submit the job and redirect to the
 				// runtime monitor
+				parameters.set(0, CliFrontend.ACTION_RUN);
 				try {
-					client.run(program, parallelism, false);
+					this.cli.parseParameters(parameters.toArray(args));
 				}
-				catch (Exception ex) {
-					LOG.error("Error submitting job to the job-manager.", ex);
+				catch (RuntimeException e) {
+					LOG.error("Error submitting job to the job-manager.", e.getCause());
 					// HACK: Is necessary because Message contains whole stack trace
-					String errorMessage = ex.getMessage().split("\n")[0];
+					String errorMessage = e.getCause().getMessage().split("\n")[0];
 					showErrorPage(resp, errorMessage);
 					return;
 				}
-				finally {
-					program.deleteExtractedLibraries();
-				}
 				resp.sendRedirect(START_PAGE_URL);
 			}
 		}
@@ -344,7 +313,7 @@ public class JobSubmissionServlet extends HttpServlet {
 
 			// submit the job
 			try {
-				Client client = new Client(config, getClass().getClassLoader());
+				Client client = new Client(GlobalConfiguration.getConfiguration(), getClass().getClassLoader());
 				client.run(job, false);
 			}
 			catch (Exception ex) {
@@ -402,8 +371,7 @@ public class JobSubmissionServlet extends HttpServlet {
 
 		PrintWriter writer = resp.getWriter();
 
-		writer
-			.println("<!DOCTYPE html PUBLIC \"-//W3C//DTD XHTML 1.0 Transitional//EN\"\n        \"http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd\">");
+		writer.println("<!DOCTYPE html PUBLIC \"-//W3C//DTD XHTML 1.0 Transitional//EN\"\n        \"http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd\">");
 		writer.println("<html>");
 		writer.println("<head>");
 		writer.println("  <title>Launch Job - Error</title>");

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java b/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java
index e30d45b..bab3cf7 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/web/WebInterfaceServer.java
@@ -25,6 +25,7 @@ import java.net.URL;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.flink.client.CliFrontend;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
@@ -59,15 +60,17 @@ public class WebInterfaceServer {
 	 * It serves the asynchronous requests for the plans and all other static resources, like
 	 * static web pages, stylesheets or javascript files.
 	 * 
+	 * @param configDir
+	 *        The path to the configuration directory.
 	 * @param config
 	 *        The configuration for the JobManager. All jobs will be sent
 	 *        to the JobManager described by this configuration.
 	 * @param port
 	 *        The port to launch the server on.
-	 * @throws IOException
-	 *         Thrown, if the server setup failed for an I/O related reason.
+	 * @throws Exception
+	 *         Thrown, if the server setup failed.
 	 */
-	public WebInterfaceServer(Configuration config, int port) throws IOException {
+	public WebInterfaceServer(String configDir, Configuration config, int port) throws Exception {
 		// if no explicit configuration is given, use the global configuration
 		if (config == null) {
 			config = GlobalConfiguration.getConfiguration();
@@ -138,12 +141,13 @@ public class WebInterfaceServer {
 												ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_PORT);
 
 		// ----- the handlers for the servlets -----
+		CliFrontend cli = new CliFrontend(configDir);
 		ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.SESSIONS);
 		servletContext.setContextPath("/");
 		servletContext.addServlet(new ServletHolder(new PactJobJSONServlet(uploadDir)), "/pactPlan");
 		servletContext.addServlet(new ServletHolder(new PlanDisplayServlet(jobManagerWebPort)), "/showPlan");
 		servletContext.addServlet(new ServletHolder(new JobsServlet(uploadDir, tmpDir, "launch.html")), "/jobs");
-		servletContext.addServlet(new ServletHolder(new JobSubmissionServlet(config, uploadDir, planDumpDir)), "/runJob");
+		servletContext.addServlet(new ServletHolder(new JobSubmissionServlet(cli, uploadDir, planDumpDir)), "/runJob");
 
 		// ----- the hander serving the written pact plans -----
 		ResourceHandler pactPlanHandler = new ResourceHandler();

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/flink-clients/src/main/resources/web-docs/js/program.js
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/resources/web-docs/js/program.js b/flink-clients/src/main/resources/web-docs/js/program.js
index d3c4c88..ea593a2 100644
--- a/flink-clients/src/main/resources/web-docs/js/program.js
+++ b/flink-clients/src/main/resources/web-docs/js/program.js
@@ -80,7 +80,6 @@ function toggleCheckboxes(box)
     var assemblerClass = box.attr('id');
 
     $('#mainCanvas').html('');
-    $('#planDescription').html('');
     pactPlanRequested = id;
 
     $.ajax({
@@ -92,7 +91,6 @@ function toggleCheckboxes(box)
   }
   else {
     $('#mainCanvas').html('');
-    $('#planplanDescription').html('');
   }
 }
 
@@ -101,20 +99,6 @@ function toggleCheckboxes(box)
  */
 function showPreviewPlan(data)
 {
-	//TODO check again the stuff below
-//  // check whether this one is still selected
-//  var active = $('.jobItemCheckbox:checked');
-//  var id = active.parentsUntil('.JobListItems').parent().attr('id').substr(4);
-//  
-//  if (pactPlanRequested == id) {
-//    if (data == undefined || data.jobname == undefined || data.jobname != pactPlanRequested || data.plan == undefined) {
-//      pactPlanRequested = 0;
-//    }
-//
-//	if(data.description != undefined) {
-//		$('#planDescription').html(data.description);
-//	}
-	
 	$("#mainCanvas").empty();
     var svgElement = "<div id=\"attach\"><svg id=\"svg-main\" width=500 height=500><g transform=\"translate(20, 20)\"/></svg></div>";
     $("#mainCanvas").append(svgElement);
@@ -216,13 +200,14 @@ function runJob ()
    
    var showPlan = $('#showPlanCheck').is(':checked');
    var suspendPlan = $('#suspendJobDuringPlanCheck').is(':checked');
+   var options = $('#commandLineOptionsField').attr('value'); //TODO? Replace with .val() ?
    var args = $('#commandLineArgsField').attr('value'); //TODO? Replace with .val() ?
    
    var url;
    if (assemblerClass == "<em>no entry class specified</em>") {
-      url = "runJob?" + $.param({ action: "submit", job: jobName, arguments: args, show_plan: showPlan, suspend: suspendPlan});
+      url = "runJob?" + $.param({ action: "submit", options: options, job: jobName, arguments: args, show_plan: showPlan, suspend: suspendPlan});
    } else {
-      url = "runJob?" + $.param({ action: "submit", job: jobName, assemblerClass: assemblerClass, arguments: args, show_plan: showPlan, suspend: suspendPlan});
+      url = "runJob?" + $.param({ action: "submit", options: options, job: jobName, assemblerClass: assemblerClass, arguments: args, show_plan: showPlan, suspend: suspendPlan});
    }
    
    window.location = url;

http://git-wip-us.apache.org/repos/asf/flink/blob/72b5dc98/flink-clients/src/main/resources/web-docs/launch.html
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/resources/web-docs/launch.html b/flink-clients/src/main/resources/web-docs/launch.html
index 7564177..e190b6a 100644
--- a/flink-clients/src/main/resources/web-docs/launch.html
+++ b/flink-clients/src/main/resources/web-docs/launch.html
@@ -1,110 +1,109 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
-        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
-<!--
-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.
--->
-<html>
-
-<head>
-  <title>Flink Query Interface</title>
-
-  <meta http-equiv="content-type" content="text/html; charset=UTF-8">
-
-  <link rel="stylesheet" type="text/css" href="css/nephelefrontend.css" />
-  <link rel="stylesheet" type="text/css" href="css/pactgraphs.css" />
-  <link rel="stylesheet" type="text/css" href="css/graph.css" />
-  <link rel="stylesheet" type="text/css" href="css/overlay.css" />
-  <link rel="stylesheet" type="text/css" href="css/bootstrap.css" /> <!-- Changed Stuff -->
-  <script type="text/javascript" src="js/jquery-2.1.0.js"></script>
-  <script type="text/javascript" src="js/graphCreator.js"></script>
-  <script type="text/javascript" src="js/d3.js" charset="utf-8"></script>
-  <script type="text/javascript" src="js/dagre-d3.js"></script>
-  <script type="text/javascript" src="js/bootstrap.min.js"></script>
-  <script type="text/javascript" src="js/jquery.tools.min.js"></script>
-  
-  
-  <script type="text/javascript" src="js/program.js"></script>
-  
-</head>
-
-<body>
-    <div class="mainHeading" style="min-width: 1225px;">
-      <h1 style="margin-top:0"><img src="img/flink-logo.png" width="100" height="100" alt="Flink Logo" />Flink Web Submission Client
-      <div style="position:absolute; top:40px; right:110px;">
-		  <button id="zoomIn" type="button" class="btn btn-default">Zoom In</button>
-		  <button id="zoomOut" type="button" class="btn btn-default">Zoom Out</button>
-		</div>
-      </h1>
-    </div>
-
-    <!-- the main panel with the jobs list and the preview pane -->
-    <div style="position: absolute; top: 110px; bottom: 210px; left: 0px; right: 0px; min-width: 1225px;">
-      <div id="jobsContents" class="boxed" style="position: absolute; top: 0px; bottom: 0px; left: 0px; width: 450px; overflow: auto;"></div>
-      <div class="boxed" style="position: absolute; top: 0px; bottom: 0px; left: 460px; right: 0px; overflow: auto;">
-        <div id="mainCanvas" class="canvas" style="position: relative; height: 100%"></div>
-      </div>
-    </div>
-
-    <!-- the footer containing the box with the run properties and the box with the upload field -->
-    <div class="footer" style="min-width: 1225px;">
-    
-      <div class="boxed" style="float: left; width: 700px; height: 200px; position: relative;">
-        <h3>Select program...</h3>
-        <div style="margin-top: 30px;">
-          <div>
-          	<div id="planDescription"></div>
-          </div>
-          <div style="text-align: right;">
-            <span class="formLabel">Arguments:</span>
-            <input id="commandLineArgsField" type="text" name="commandLine" style="width: 580px;"/>
-          </div>
-          <div class="spaced" style="margin-left: 20px;">
-            <input type="checkbox" id="showPlanCheck" checked="checked"><span class="formLabel">Show optimizer plan</span>
-            <div id="suspendOption"><span>&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;</span><input type="checkbox" id="suspendJobDuringPlanCheck" checked="checked"><span class="formLabel">Suspend execution while showing plan</span></div>
-          </div>
-        </div>
-        <div class="footer" style="text-align: right;">
-          <span id="run_error_text" class="error_text"></span>
-          <input id="run_button" type="button" value="Run Job" style="width: 75px; margin-left: 100px;"/>
-        </div>
-      </div>
-      
-      <div class="boxed" style="float: left; width: 500px; height: 150px; position: relative;">
-        <h3>Select a new program to upload...</h3>
-        <form id="upload_form" action="jobs" enctype="multipart/form-data" method="POST">
-          <div style="position: relative; margin-top: 30px;">
-             <input id="upload_file_name_text" type="text" name="file_name" disabled="disabled" style="position: absolute; top: 0px; right: 85px; width: 380px; z-index: 3;"/>
-             <input type="button" value="Browse" style="width: 75px; position: absolute; right: 0px; top: 0px; z-index: 1;" />
-             <input id="upload_file_input" class="translucent" type="file" name="upload_jar_file" style="position: absolute; right: 0px; top: 0px; height: 30px; width=75px; z-index: 2;" />
-          </div>
-        </form>
-        <div class="footer" style="text-align: right;">
-          <span id="upload_error_text" class="error_text"></span>
-          <input id="upload_submit_button" type="button" value="Upload" style="width: 75px; margin-left: 100px;"/>
-        </div>
-      </div>
-      
-    </div>
-    
-    <!-- the overlay -->
-    <div class="simple_overlay" id="propertyO">
-	  <div id="propertyCanvas" class="propertyCanvas">
-	  </div>
-	</div>
-</body>
-</html>
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
+        "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<!--
+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.
+-->
+<html>
+
+<head>
+  <title>Flink Query Interface</title>
+
+  <meta http-equiv="content-type" content="text/html; charset=UTF-8">
+
+  <link rel="stylesheet" type="text/css" href="css/nephelefrontend.css" />
+  <link rel="stylesheet" type="text/css" href="css/pactgraphs.css" />
+  <link rel="stylesheet" type="text/css" href="css/graph.css" />
+  <link rel="stylesheet" type="text/css" href="css/overlay.css" />
+  <link rel="stylesheet" type="text/css" href="css/bootstrap.css" /> <!-- Changed Stuff -->
+
+  <script type="text/javascript" src="js/jquery-2.1.0.js"></script>
+  <script type="text/javascript" src="js/graphCreator.js"></script>
+  <script type="text/javascript" src="js/d3.js" charset="utf-8"></script>
+  <script type="text/javascript" src="js/dagre-d3.js"></script>
+  <script type="text/javascript" src="js/bootstrap.min.js"></script>
+  <script type="text/javascript" src="js/jquery.tools.min.js"></script>
+  <script type="text/javascript" src="js/program.js"></script>
+</head>
+
+<body>
+    <div class="mainHeading" style="min-width: 1225px;">
+      <h1 style="margin-top:0"><img src="img/flink-logo.png" width="100" height="100" alt="Flink Logo" />Flink Web Submission Client
+        <div style="position:absolute; top:40px; right:110px;">
+          <button id="zoomIn" type="button" class="btn btn-default">Zoom In</button>
+          <button id="zoomOut" type="button" class="btn btn-default">Zoom Out</button>
+        </div>
+      </h1>
+    </div>
+
+    <!-- the main panel with the jobs list and the preview pane -->
+    <div style="position: absolute; top: 110px; bottom: 210px; left: 0px; right: 0px; min-width: 1225px;">
+      <div id="jobsContents" class="boxed" style="position: absolute; top: 0px; bottom: 0px; left: 0px; width: 450px; overflow: auto;"></div>
+      <div class="boxed" style="position: absolute; top: 0px; bottom: 0px; left: 460px; right: 0px; overflow: auto;">
+        <div id="mainCanvas" class="canvas" style="position: relative; height: 100%"></div>
+      </div>
+    </div>
+
+    <!-- the footer containing the box with the run properties and the box with the upload field -->
+    <div class="footer" style="min-width: 1225px;">
+
+      <div class="boxed" style="float: left; width: 700px; height: 200px; position: relative;">
+        <h3>Select program...</h3>
+        <div style="margin-top: 30px;">
+          <div style="text-align: right;">
+            <span class="formLabel">Flink Options:</span>
+            <input id="commandLineOptionsField" type="text" name="commandLineOptions" style="width: 520px;"/>
+          </div>
+          <div style="text-align: right; margin-top: 5px;">
+            <span class="formLabel">Program Arguments:</span>
+            <input id="commandLineArgsField" type="text" name="commandLineArguments" style="width: 520px;"/>
+          </div>
+          <div class="spaced" style="margin-left: 20px;">
+            <input type="checkbox" id="showPlanCheck" checked="checked"><span class="formLabel">Show optimizer plan</span>
+            <div id="suspendOption"><span>&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;</span><input type="checkbox" id="suspendJobDuringPlanCheck" checked="checked"><span class="formLabel">Suspend execution while showing plan</span></div>
+          </div>
+        </div>
+        <div class="footer" style="text-align: right;">
+          <span id="run_error_text" class="error_text"></span>
+          <input id="run_button" type="button" value="Run Job" style="width: 75px; margin-left: 100px;"/>
+        </div>
+      </div>
+
+      <div class="boxed" style="float: left; width: 500px; height: 150px; position: relative;">
+        <h3>Select a new program to upload...</h3>
+        <form id="upload_form" action="jobs" enctype="multipart/form-data" method="POST">
+          <div style="position: relative; margin-top: 30px;">
+             <input id="upload_file_name_text" type="text" name="file_name" disabled="disabled" style="position: absolute; top: 0px; right: 85px; width: 380px; z-index: 3;"/>
+             <input type="button" value="Browse" style="width: 75px; position: absolute; right: 0px; top: 0px; z-index: 1;" />
+             <input id="upload_file_input" class="translucent" type="file" name="upload_jar_file" style="position: absolute; right: 0px; top: 0px; height: 30px; width=75px; z-index: 2;" />
+          </div>
+        </form>
+        <div class="footer" style="text-align: right;">
+          <span id="upload_error_text" class="error_text"></span>
+          <input id="upload_submit_button" type="button" value="Upload" style="width: 75px; margin-left: 100px;"/>
+        </div>
+      </div>
+
+    </div>
+
+    <!-- the overlay -->
+    <div class="simple_overlay" id="propertyO">
+      <div id="propertyCanvas" class="propertyCanvas">
+    </div>
+  </div>
+</body>
+</html>