You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/10/17 21:03:25 UTC

[05/21] flink git commit: [FLINK-2844] [web frontend] Remove old web interface

http://git-wip-us.apache.org/repos/asf/flink/blob/df448625/flink-runtime/src/main/resources/web-docs-infoserver/taskmanagers.html
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/resources/web-docs-infoserver/taskmanagers.html b/flink-runtime/src/main/resources/web-docs-infoserver/taskmanagers.html
deleted file mode 100644
index c7cb52b..0000000
--- a/flink-runtime/src/main/resources/web-docs-infoserver/taskmanagers.html
+++ /dev/null
@@ -1,182 +0,0 @@
-<!DOCTYPE html>
-<!--
-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 lang="en">
-  <head>
-    <meta charset="utf-8">
-    <meta name="viewport" content="width=device-width, initial-scale=1.0">
-    <meta name="description" content="">
-    <meta name="author" content="">
-
-    <title>Dashboard - Apache Flink</title>
-
-    <!-- Bootstrap core CSS -->
-    <link href="css/bootstrap.css" rel="stylesheet">
-
-    <!-- Add custom CSS here -->
-    <link href="css/sb-admin.css" rel="stylesheet">
-    <link rel="stylesheet" href="font-awesome/css/font-awesome.min.css">
-    
-    <!-- Page Specific CSS -->
-    <link rel="stylesheet" type="text/css" href="css/nephelefrontend.css" />
-    
-    <!-- Scripts from Bootstrap -->
-    <script src="js/jquery-2.1.0.js"></script>
-    <script src="js/bootstrap.js"></script>
-
-    <!-- Scripts and CSS for the visualization -->
-    <link rel="stylesheet" href="css/rickshaw.min.css">
-    <script src="js/d3.min.js"></script>
-    <script src="js/d3.layout.min.js"></script>
-    <script src="js/rickshaw.min.js"></script>
-
-    <style>
-    .chart_container {
-        position: relative;
-        font-family: Arial, Helvetica, sans-serif;
-        display: none;
-    }
-    .chart {
-        position: relative;
-        left: 65px;
-    }
-    .y_axis {
-        position: absolute;
-        top: 0;
-        bottom: 0;
-        width: 60px;
-    }
-    .y_axis-load {
-        position: absolute;
-        top: 0;
-        bottom: 0;
-        width: 60px;
-        right: 0px;
-    }
-    .axis_label {
-        position:absolute;
-        text-align: center;
-        width: inherit;
-    }
-
-    .legend {
-        display: inline-block;
-        vertical-align: top;
-        margin: -50px 0 0 10px;
-    }
-    </style>
-
-    <!-- Scripts from Flink -->
-	<script type="text/javascript" src="js/helpers.js"></script>
-	<script type="text/javascript" src="js/taskmanager.js"></script>
-	<script type="text/javascript" src="js/jcanvas.min.js"></script>
-	
-	<!-- Load Menu -->
-	<script type="text/javascript">
-	$(document).ready(function() {
-		$.ajax({ url : "menu?get=taskmanagers", type : "GET", cache: false, success : function(html) {
-			$("#side-menu").empty();
-			$("#side-menu").append(html);
-		}, dataType : "html"
-		});
-	});
-  	</script>
-  </head>
-
-  <body>
-
-    <div id="wrapper">
-	  
-      <!-- Sidebar -->
-	  <nav class='navbar navbar-inverse navbar-fixed-top' role='navigation'>
-        <!-- Brand and toggle get grouped for better mobile display -->
-        <div class="navbar-header">
-          <button type="button" class="navbar-toggle" data-toggle="collapse" data-target=".navbar-ex1-collapse">
-            <span class="sr-only">Toggle navigation</span>
-            <span class="icon-bar"></span>
-            <span class="icon-bar"></span>
-            <span class="icon-bar"></span>
-          </button>
-          <table>
-            <tr>
-              <td><img src="img/flink-logo.png" alt="Flink Logo" style="margin-left:15px; margin-top:5px; width:40px;height:40px";></td>
-              <td style="vertical-align:top"><a class="navbar-brand" href="index.html">Apache Flink</a></td>
-            </tr>
-          </table>
-        </div>
-	 
-        <!-- Collect the nav links, forms, and other content for toggling -->
-        <div class="collapse navbar-collapse navbar-ex1-collapse">
-          <ul id="side-menu" class="nav navbar-nav side-nav"> 
-          <!-- Filled via script -->
-          </ul>
-          <ul class="nav navbar-nav navbar-right navbar-user">
-            <li class="dropdown user-dropdown">
-              <a href="#" class="dropdown-toggle" data-toggle="dropdown"><i class="fa fa-archive"></i> Log Files<b class="caret"></b></a>
-              <ul class="dropdown-menu">
-                <li><a href="logInfo"><i class="fa fa-keyboard-o"></i> Log Info</a></li>
-                <li><a href="logInfo?get=stdout"><i class="fa fa-keyboard-o"></i> Stdout</a></li>
-              </ul>
-            </li>
-          </ul>
-        </div><!-- /.navbar-collapse -->
-      </nav>
-
-      <div id="page-wrapper">
-
-        <div class="row">
-          <div class="col-lg-12">
-            <h1 id="page-title">Task Managers</h1>
-            <ol class="breadcrumb">
-              <li><a href="index.html"><i class="icon-dashboard"></i> Dashboard</a></li>
-              <li class="active"><i class="icon-file-alt"></i>Task Managers</li>
-            </ol>
-          </div>
-          <div class="col-lg-12" id="metricsControl" style="padding-bottom:25px;">
-              <button id="metrics-limit-3" onclick="updateLimit(this)" class="btn btn-default active">Show metrics for 3 TaskManagers</button>
-              <button id="metrics-limit-all" onclick="updateLimit(this)" class="btn btn-default">Show metrics for all TaskManagers</button>
-              <button id="metrics-limit-none" onclick="updateLimit(this)" class="btn btn-default">Disable metrics</button>
-          </div>
-
-		  <div class="col-lg-12">
-	          <div class="table-responsive" id="taskmanagerTable">
-                  <table class="table table-bordered table-hover table-striped">
-                      <tr id="taskmanagerTable-header"><th>TaskManager</th>
-                          <th id="tmTableHeaderMemStat">Memory Statistics</th>
-                          <th>Information</th>
-                      </tr>
-                  </table>
-	          </div>
-	      </div>
-          <div class="col-lg-12" style="display:none">
-              <i class="fa fa-times" style="position:absolute; right:20px; top:20px;" onclick="$(this).parent().hide();"></i>
-              <div id="allMetrics"></div>
-          </div>
-          <div class="col-lg-12" style="display:none">
-              <i class="fa fa-times" style="position:absolute; right:20px; top:20px;" onclick="$(this).parent().hide();"></i>
-              <div id="taskManagerStackTrace"></div>
-          </div>
-        </div><!-- /.row -->
-
-      </div><!-- /#page-wrapper -->
-
-    </div><!-- /#wrapper -->
-
-  </body>
-</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/df448625/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
index d1031a8..95637bb 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.jobmanager
 import java.io.{File, IOException}
 import java.lang.reflect.{InvocationTargetException, Constructor}
 import java.net.InetSocketAddress
-import java.util.{UUID, Collections}
+import java.util.UUID
 
 import akka.actor.Status.Failure
 import akka.actor.{Props, Terminated, PoisonPill, ActorRef, ActorSystem}
@@ -37,7 +37,6 @@ import org.apache.flink.runtime.blob.BlobServer
 import org.apache.flink.runtime.client._
 import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex}
 import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator
-import org.apache.flink.runtime.jobmanager.web.WebInfoServer
 import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService}
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
 import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph
@@ -1367,21 +1366,13 @@ object JobManager {
 
         val leaderRetrievalService = StandaloneUtils.createLeaderRetrievalService(configuration)
 
-        // start the job manager web frontend
-        val webServer = if (
-          configuration.getBoolean(
-            ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY,
-            false)) {
-
-          LOG.info("Starting NEW JobManger web frontend")
-          // start the new web frontend. we need to load this dynamically
-          // because it is not in the same project/dependencies
-          startWebRuntimeMonitor(configuration, leaderRetrievalService, jobManagerSystem)
-        }
-        else {
-          LOG.info("Starting JobManger web frontend")
-          new WebInfoServer(configuration, leaderRetrievalService, jobManagerSystem)
-        }
+        LOG.info("Starting JobManger web frontend")
+        // start the web frontend. we need to load this dynamically
+        // because it is not in the same project/dependencies
+        val webServer = WebMonitorUtils.startWebRuntimeMonitor(
+          configuration,
+          leaderRetrievalService,
+          jobManagerSystem)
 
         if(webServer != null) {
           webServer.start()
@@ -1817,50 +1808,4 @@ object JobManager {
     val timeout = AkkaUtils.getLookupTimeout(config)
     getJobManagerActorRef(address, system, timeout)
   }
-
-
-  // --------------------------------------------------------------------------
-  //  Utilities
-  // --------------------------------------------------------------------------
-
-  /**
-   * Starts the web runtime monitor. Because the actual implementation of the
-   * runtime monitor is in another project, we load the runtime monitor dynamically.
-   * 
-   * Because failure to start the web runtime monitor is not considered fatal,
-   * this method does not throw any exceptions, but only logs them.
-   * 
-   * @param config The configuration for the runtime monitor.
-   * @param leaderRetrievalService Leader retrieval service to get the leading JobManager
-   */
-  def startWebRuntimeMonitor(
-      config: Configuration,
-      leaderRetrievalService: LeaderRetrievalService,
-      actorSystem: ActorSystem)
-    : WebMonitor = {
-    // try to load and instantiate the class
-    try {
-      val classname = "org.apache.flink.runtime.webmonitor.WebRuntimeMonitor"
-      val clazz: Class[_ <: WebMonitor] = Class.forName(classname)
-        .asSubclass(classOf[WebMonitor])
-
-      val ctor: Constructor[_ <: WebMonitor] = clazz.getConstructor(classOf[Configuration],
-        classOf[LeaderRetrievalService],
-        classOf[ActorSystem])
-      ctor.newInstance(config, leaderRetrievalService, actorSystem)
-    }
-    catch {
-      case e: ClassNotFoundException =>
-        LOG.error("Could not load web runtime monitor. " +
-          "Probably reason: flink-runtime-web is not in the classpath")
-        LOG.debug("Caught exception", e)
-        null
-      case e: InvocationTargetException =>
-        LOG.error("WebServer could not be created", e.getTargetException())
-        null
-      case t: Throwable =>
-        LOG.error("Failed to instantiate web runtime monitor.", t)
-        null
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/df448625/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
index 77e977f..8ccc66d 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
@@ -34,13 +34,12 @@ import org.apache.flink.runtime.akka.AkkaUtils
 import org.apache.flink.runtime.client.{JobExecutionException, JobClient}
 import org.apache.flink.runtime.instance.{AkkaActorGateway, ActorGateway}
 import org.apache.flink.runtime.jobgraph.JobGraph
-import org.apache.flink.runtime.jobmanager.web.WebInfoServer
 import org.apache.flink.runtime.jobmanager.{JobManager, RecoveryMode}
 import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalService, LeaderRetrievalListener,
 StandaloneLeaderRetrievalService}
 import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtAnyJobManager
-import org.apache.flink.runtime.util.{StandaloneUtils, ZooKeeperUtils}
-import org.apache.flink.runtime.webmonitor.WebMonitor
+import org.apache.flink.runtime.util.ZooKeeperUtils
+import org.apache.flink.runtime.webmonitor.{WebMonitorUtils, WebMonitor}
 
 import org.slf4j.LoggerFactory
 
@@ -289,20 +288,11 @@ abstract class FlinkMiniCluster(
       // TODO: Add support for HA: Make web server work independently from the JM
       val leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManagerAkkaURL)
 
-      // start the job manager web frontend
-      val webServer = if (
-        config.getBoolean(
-          ConfigConstants.JOB_MANAGER_NEW_WEB_FRONTEND_KEY,
-          false)) {
-
-        LOG.info("Starting NEW JobManger web frontend")
-        // start the new web frontend. we need to load this dynamically
-        // because it is not in the same project/dependencies
-        JobManager.startWebRuntimeMonitor(config, leaderRetrievalService, actorSystem)
-      } else {
-        LOG.info("Starting JobManger web frontend")
-        new WebInfoServer(config, leaderRetrievalService, actorSystem)
-      }
+      LOG.info("Starting JobManger web frontend")
+      // start the new web frontend. we need to load this dynamically
+      // because it is not in the same project/dependencies
+      val webServer = WebMonitorUtils.startWebRuntimeMonitor(
+        config, leaderRetrievalService, actorSystem)
 
       webServer.start()
 

http://git-wip-us.apache.org/repos/asf/flink/blob/df448625/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index b4bddef..0dd20b1 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -57,7 +57,14 @@ under the License.
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
-		
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime-web</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-clients</artifactId>

http://git-wip-us.apache.org/repos/asf/flink/blob/df448625/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
index 2ca35ed..df13bdf 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
@@ -18,9 +18,9 @@
 package org.apache.flink.test.web;
 
 
-import org.apache.commons.io.FileUtils;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.runtime.webmonitor.WebMonitor;
+import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
 import org.apache.flink.test.util.MultipleProgramsTestBase;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.codehaus.jettison.json.JSONArray;
@@ -33,9 +33,9 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import scala.concurrent.duration.FiniteDuration;
 
-import java.io.File;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 @RunWith(Parameterized.class)
@@ -68,30 +68,25 @@ public class WebFrontendITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void getNumberOfTaskManagers() {
+	public void getFrontPage() {
 		try {
-			Assert.assertEquals("{\"taskmanagers\": "+cluster.getTaskManagers().size()+", \"slots\": 4}",
-					TestBaseUtils.getFromHTTP("http://localhost:" + port + "/jobsInfo?get=taskmanagers"));
-		}catch(Throwable e) {
+			String fromHTTP = TestBaseUtils.getFromHTTP("http://localhost:" + port + "/index.html");
+			String text = "Apache Flink Dashboard";
+			Assert.assertTrue("Startpage should contain " + text, fromHTTP.contains(text));
+		} catch (Exception e) {
 			e.printStackTrace();
 			Assert.fail(e.getMessage());
 		}
 	}
 
 	@Test
-	public void getTaskmanagers() {
+	public void getNumberOfTaskManagers() {
 		try {
-			String json = getFromHTTP("http://localhost:" + port + "/setupInfo?get=taskmanagers");
-			JSONObject parsed = new JSONObject(json);
-			Object taskManagers = parsed.get("taskmanagers");
+			String json = TestBaseUtils.getFromHTTP("http://localhost:" + port + "/taskmanagers/");
+			JSONObject response = new JSONObject(json);
+			JSONArray taskManagers = response.getJSONArray("taskmanagers");
 			Assert.assertNotNull(taskManagers);
-			Assert.assertTrue(taskManagers instanceof JSONArray);
-			JSONArray tma = (JSONArray) taskManagers;
-			Assert.assertEquals(cluster.numTaskManagers(), tma.length());
-			Object taskManager = tma.get(0);
-			Assert.assertNotNull(taskManager);
-			Assert.assertTrue(taskManager instanceof JSONObject);
-			Assert.assertEquals(4, ((JSONObject) taskManager).getInt("freeSlots"));
+			Assert.assertEquals(cluster.numTaskManagers(), taskManagers.length());
 		}catch(Throwable e) {
 			e.printStackTrace();
 			Assert.fail(e.getMessage());
@@ -99,28 +94,51 @@ public class WebFrontendITCase extends MultipleProgramsTestBase {
 	}
 
 	@Test
-	public void getLogfiles() {
+	public void getTaskmanagers() {
 		try {
-			String logPath = cluster.configuration().getString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, null);
-			Assert.assertNotNull(logPath);
-			FileUtils.writeStringToFile(new File(logPath, "jobmanager-main.log"), "test content");
-
-			String logs = getFromHTTP("http://localhost:8081/logInfo");
-			Assert.assertTrue(logs.contains("test content"));
+			String json = getFromHTTP("http://localhost:" + port + "/taskmanagers/");
+			JSONObject parsed = new JSONObject(json);
+			JSONArray taskManagers = parsed.getJSONArray("taskmanagers");
+			Assert.assertNotNull(taskManagers);
+			Assert.assertEquals(cluster.numTaskManagers(), taskManagers.length());
+			JSONObject taskManager = taskManagers.getJSONObject(0);
+			Assert.assertNotNull(taskManager);
+			Assert.assertEquals(4, taskManager.getInt("freeSlots"));
 		}catch(Throwable e) {
 			e.printStackTrace();
 			Assert.fail(e.getMessage());
 		}
 	}
 
+// TODO activate this test after logging retrieval has been added to the new web frontend
+//	@Test
+//	public void getLogfiles() {
+//		try {
+//			String logPath = cluster.configuration().getString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, null);
+//			Assert.assertNotNull(logPath);
+//			FileUtils.writeStringToFile(new File(logPath, "jobmanager-main.log"), "test content");
+//
+//			String logs = getFromHTTP("http://localhost:8081/logInfo");
+//			Assert.assertTrue(logs.contains("test content"));
+//		}catch(Throwable e) {
+//			e.printStackTrace();
+//			Assert.fail(e.getMessage());
+//		}
+//	}
+
 	@Test
 	public void getConfiguration() {
 		try {
-			String config = getFromHTTP("http://localhost:" + port + "/setupInfo?get=globalC");
-			JSONObject parsed = new JSONObject(config);
-			Assert.assertEquals(logDir.toString(), parsed.getString("jobmanager.web.logpath"));
-			Assert.assertEquals(cluster.configuration().getString("taskmanager.numberOfTaskSlots", null), parsed.getString("taskmanager.numberOfTaskSlots"));
-		}catch(Throwable e) {
+			String config = getFromHTTP("http://localhost:" + port + "/jobmanager/config");
+			JSONArray array = new JSONArray(config);
+
+			Map<String, String> conf = WebMonitorUtils.fromKeyValueJsonArray(array);
+			Assert.assertEquals(logDir.toString(),
+					conf.get(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY));
+			Assert.assertEquals(
+					cluster.configuration().getString("taskmanager.numberOfTaskSlots", null),
+					conf.get(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS));
+		} catch(Throwable e) {
 			e.printStackTrace();
 			Assert.fail(e.getMessage());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/df448625/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index c2e9a45..b082f6a 100644
--- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -23,6 +23,7 @@ import org.apache.flink.client.FlinkYarnSessionCli;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.client.JobClient;
+import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnClient;
 import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster;
 import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus;
@@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.log4j.Level;
+import org.codehaus.jettison.json.JSONArray;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.After;
 import org.junit.Assert;
@@ -196,14 +198,20 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 			}
 			LOG.info("Got application URL from YARN {}", url);
 
-			// get number of TaskManagers:
-			Assert.assertEquals("{\"taskmanagers\": 1, \"slots\": 1}", TestBaseUtils.getFromHTTP(url + "jobsInfo?get=taskmanagers"));
+			String response = TestBaseUtils.getFromHTTP(url + "taskmanagers/");
+			JSONObject parsedTMs = new JSONObject(response);
+			JSONArray taskManagers = parsedTMs.getJSONArray("taskmanagers");
+			Assert.assertNotNull(taskManagers);
+			Assert.assertEquals(1, taskManagers.length());
+			Assert.assertEquals(1, taskManagers.getJSONObject(0).getInt("slotsNumber"));
 
 			// get the configuration from webinterface & check if the dynamic properties from YARN show up there.
-			String config = TestBaseUtils.getFromHTTP(url + "setupInfo?get=globalC");
-			JSONObject parsed = new JSONObject(config);
-			Assert.assertEquals("veryFancy", parsed.getString("fancy-configuration-value"));
-			Assert.assertEquals("3", parsed.getString("yarn.maximum-failed-containers"));
+			String jsonConfig = TestBaseUtils.getFromHTTP(url + "jobmanager/config");
+			JSONArray parsed = new JSONArray(jsonConfig);
+			Map<String, String> parsedConfig = WebMonitorUtils.fromKeyValueJsonArray(parsed);
+
+			Assert.assertEquals("veryFancy", parsedConfig.get("fancy-configuration-value"));
+			Assert.assertEquals("3", parsedConfig.get("yarn.maximum-failed-containers"));
 
 			// -------------- FLINK-1902: check if jobmanager hostname/port are shown in web interface
 			// first, get the hostname/port
@@ -218,12 +226,15 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 			}
 			LOG.info("Extracted hostname:port: {} {}", hostname, port);
 
-			Assert.assertEquals("unable to find hostname in " + parsed, hostname, parsed.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY).toLowerCase());
-			Assert.assertEquals("unable to find port in " + parsed, port, parsed.getString(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY));
+			Assert.assertEquals("unable to find hostname in " + parsed, hostname,
+					parsedConfig.get(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY));
+			Assert.assertEquals("unable to find port in " + parsed, port,
+					parsedConfig.get(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY));
 
+			// TODO disabled until log files available in new web frontend
 			// test logfile access
-			String logs = TestBaseUtils.getFromHTTP(url + "logInfo");
-			Assert.assertTrue(logs.contains("Starting YARN ApplicationMaster/JobManager (Version"));
+			//String logs = TestBaseUtils.getFromHTTP(url + "logInfo");
+			//Assert.assertTrue(logs.contains("Starting YARN ApplicationMaster/JobManager (Version"));
 		} catch(Throwable e) {
 			LOG.warn("Error while running test",e);
 			Assert.fail(e.getMessage());

http://git-wip-us.apache.org/repos/asf/flink/blob/df448625/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 63ffa55..efe2101 100644
--- a/pom.xml
+++ b/pom.xml
@@ -178,28 +178,6 @@ under the License.
 		-->
 		<dependencies>
 
-			<!-- Make sure we use a consistent jetty version throughout the project -->
-			<dependency>
-				<groupId>org.eclipse.jetty</groupId>
-				<artifactId>jetty-server</artifactId>
-				<version>8.0.0.M1</version>
-			</dependency>
-			<dependency>
-				<groupId>org.eclipse.jetty</groupId>
-				<artifactId>jetty-security</artifactId>
-				<version>8.0.0.M1</version>
-			</dependency>
-			<dependency>
-				<groupId>org.eclipse.jetty</groupId>
-				<artifactId>jetty-servlet</artifactId>
-				<version>8.0.0.M1</version>
-			</dependency>
-			<dependency>
-				<groupId>org.eclipse.jetty</groupId>
-				<artifactId>jetty-util</artifactId>
-				<version>8.0.0.M1</version>
-			</dependency>
-
 			<!-- Make sure we use a consistent avro version throughout the project -->
 			<dependency>
 				<groupId>org.apache.avro</groupId>
@@ -746,14 +724,6 @@ under the License.
 						<exclude>**/resources/**/jquery*</exclude>
 						<exclude>**/resources/**/bootstrap*</exclude>
 						<exclude>flink-clients/src/main/resources/web-docs/js/*d3.js</exclude>
-						<exclude>flink-runtime/src/main/resources/web-docs-infoserver/css/sb-admin.css</exclude>
-						<exclude>flink-runtime/src/main/resources/web-docs-infoserver/js/flot/*</exclude>
-						<exclude>flink-runtime/src/main/resources/web-docs-infoserver/js/jcanvas.min.js</exclude>
-						<exclude>flink-runtime/src/main/resources/web-docs-infoserver/js/timeline.js</exclude>
-						<exclude>flink-runtime/src/main/resources/web-docs-infoserver/js/d3.min.js</exclude>
-						<exclude>flink-runtime/src/main/resources/web-docs-infoserver/js/d3.layout.min.js</exclude>
-						<exclude>flink-runtime/src/main/resources/web-docs-infoserver/js/rickshaw.*</exclude>
-						<exclude>flink-runtime/src/main/resources/web-docs-infoserver/css/rickshaw.*</exclude>
 
 						<!-- web dashboard config JSON files -->
 						<exclude>flink-runtime-web/web-dashboard/package.json</exclude>