You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ir...@apache.org on 2015/05/21 04:14:06 UTC

spark git commit: [SPARK-7750] [WEBUI] Rename endpoints from `json` to `api` to allow fu…

Repository: spark
Updated Branches:
  refs/heads/master 5196efff5 -> a70bf06b7


[SPARK-7750] [WEBUI] Rename endpoints from `json` to `api` to allow fu…

…rther extension to non-json outputs too.

Author: Hari Shreedharan <hs...@apache.org>

Closes #6273 from harishreedharan/json-to-api and squashes the following commits:

e14b73b [Hari Shreedharan] Rename `getJsonServlet` to `getServletHandler` i
42f8acb [Hari Shreedharan] Import order fixes.
2ef852f [Hari Shreedharan] [SPARK-7750][WebUI] Rename endpoints from `json` to `api` to allow further extension to non-json outputs too.


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

Branch: refs/heads/master
Commit: a70bf06b790add5f279a69607df89ed36155b0e4
Parents: 5196eff
Author: Hari Shreedharan <hs...@apache.org>
Authored: Wed May 20 21:13:10 2015 -0500
Committer: Imran Rashid <ir...@cloudera.com>
Committed: Wed May 20 21:13:10 2015 -0500

----------------------------------------------------------------------
 .../spark/deploy/history/HistoryServer.scala    |   5 +-
 .../spark/deploy/master/ui/MasterWebUI.scala    |   5 +-
 .../spark/status/api/v1/ApiRootResource.scala   | 255 +++++++++++++++++++
 .../spark/status/api/v1/JsonRootResource.scala  | 255 -------------------
 .../scala/org/apache/spark/ui/SparkUI.scala     |   5 +-
 .../deploy/history/HistoryServerSuite.scala     |   4 +-
 .../org/apache/spark/ui/UISeleniumSuite.scala   |  16 +-
 docs/monitoring.md                              |  10 +-
 8 files changed, 279 insertions(+), 276 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a70bf06b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index 517cbe5..5a0eb58 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -25,7 +25,8 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
 
 import org.apache.spark.{Logging, SecurityManager, SparkConf}
 import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, JsonRootResource, UIRoot}
+import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource,
+  UIRoot}
 import org.apache.spark.ui.{SparkUI, UIUtils, WebUI}
 import org.apache.spark.ui.JettyUtils._
 import org.apache.spark.util.{SignalLogger, Utils}
@@ -125,7 +126,7 @@ class HistoryServer(
   def initialize() {
     attachPage(new HistoryPage(this))
 
-    attachHandler(JsonRootResource.getJsonServlet(this))
+    attachHandler(ApiRootResource.getServletHandler(this))
 
     attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a70bf06b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index eb26e9f..2111a85 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -19,7 +19,8 @@ package org.apache.spark.deploy.master.ui
 
 import org.apache.spark.Logging
 import org.apache.spark.deploy.master.Master
-import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, JsonRootResource, UIRoot}
+import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationsListResource, ApplicationInfo,
+  UIRoot}
 import org.apache.spark.ui.{SparkUI, WebUI}
 import org.apache.spark.ui.JettyUtils._
 import org.apache.spark.util.RpcUtils
@@ -47,7 +48,7 @@ class MasterWebUI(val master: Master, requestedPort: Int)
     attachPage(new HistoryNotFoundPage(this))
     attachPage(masterPage)
     attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"))
-    attachHandler(JsonRootResource.getJsonServlet(this))
+    attachHandler(ApiRootResource.getServletHandler(this))
     attachHandler(createRedirectHandler(
       "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST")))
     attachHandler(createRedirectHandler(

http://git-wip-us.apache.org/repos/asf/spark/blob/a70bf06b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
new file mode 100644
index 0000000..bf2cc2e
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
@@ -0,0 +1,255 @@
+/*
+ * 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.spark.status.api.v1
+
+import javax.servlet.ServletContext
+import javax.ws.rs._
+import javax.ws.rs.core.{Context, Response}
+
+import com.sun.jersey.api.core.ResourceConfig
+import com.sun.jersey.spi.container.servlet.ServletContainer
+import org.eclipse.jetty.server.handler.ContextHandler
+import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
+
+import org.apache.spark.SecurityManager
+import org.apache.spark.ui.SparkUI
+
+/**
+ * Main entry point for serving spark application metrics as json, using JAX-RS.
+ *
+ * Each resource should have endpoints that return **public** classes defined in api.scala.  Mima
+ * binary compatibility checks ensure that we don't inadvertently make changes that break the api.
+ * The returned objects are automatically converted to json by jackson with JacksonMessageWriter.
+ * In addition, there are a number of tests in HistoryServerSuite that compare the json to "golden
+ * files".  Any changes and additions should be reflected there as well -- see the notes in
+ * HistoryServerSuite.
+ */
+@Path("/v1")
+private[v1] class ApiRootResource extends UIRootFromServletContext {
+
+  @Path("applications")
+  def getApplicationList(): ApplicationListResource = {
+    new ApplicationListResource(uiRoot)
+  }
+
+  @Path("applications/{appId}")
+  def getApplication(): OneApplicationResource = {
+    new OneApplicationResource(uiRoot)
+  }
+
+  @Path("applications/{appId}/{attemptId}/jobs")
+  def getJobs(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): AllJobsResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new AllJobsResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/jobs")
+  def getJobs(@PathParam("appId") appId: String): AllJobsResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new AllJobsResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/jobs/{jobId: \\d+}")
+  def getJob(@PathParam("appId") appId: String): OneJobResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new OneJobResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/jobs/{jobId: \\d+}")
+  def getJob(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): OneJobResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new OneJobResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/executors")
+  def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new ExecutorListResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/executors")
+  def getExecutors(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): ExecutorListResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new ExecutorListResource(ui)
+    }
+  }
+
+
+  @Path("applications/{appId}/stages")
+  def getStages(@PathParam("appId") appId: String): AllStagesResource= {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new AllStagesResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/stages")
+  def getStages(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): AllStagesResource= {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new AllStagesResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/stages/{stageId: \\d+}")
+  def getStage(@PathParam("appId") appId: String): OneStageResource= {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new OneStageResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/stages/{stageId: \\d+}")
+  def getStage(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): OneStageResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new OneStageResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/storage/rdd")
+  def getRdds(@PathParam("appId") appId: String): AllRDDResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new AllRDDResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/storage/rdd")
+  def getRdds(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): AllRDDResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new AllRDDResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/storage/rdd/{rddId: \\d+}")
+  def getRdd(@PathParam("appId") appId: String): OneRDDResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new OneRDDResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/storage/rdd/{rddId: \\d+}")
+  def getRdd(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): OneRDDResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new OneRDDResource(ui)
+    }
+  }
+
+}
+
+private[spark] object ApiRootResource {
+
+  def getServletHandler(uiRoot: UIRoot): ServletContextHandler = {
+    val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS)
+    jerseyContext.setContextPath("/api")
+    val holder:ServletHolder = new ServletHolder(classOf[ServletContainer])
+    holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
+      "com.sun.jersey.api.core.PackagesResourceConfig")
+    holder.setInitParameter("com.sun.jersey.config.property.packages",
+      "org.apache.spark.status.api.v1")
+    holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS,
+      classOf[SecurityFilter].getCanonicalName)
+    UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot)
+    jerseyContext.addServlet(holder, "/*")
+    jerseyContext
+  }
+}
+
+/**
+ * This trait is shared by the all the root containers for application UI information --
+ * the HistoryServer, the Master UI, and the application UI.  This provides the common
+ * interface needed for them all to expose application info as json.
+ */
+private[spark] trait UIRoot {
+  def getSparkUI(appKey: String): Option[SparkUI]
+  def getApplicationInfoList: Iterator[ApplicationInfo]
+
+  /**
+   * Get the spark UI with the given appID, and apply a function
+   * to it.  If there is no such app, throw an appropriate exception
+   */
+  def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = {
+    val appKey = attemptId.map(appId + "/" + _).getOrElse(appId)
+    getSparkUI(appKey) match {
+      case Some(ui) =>
+        f(ui)
+      case None => throw new NotFoundException("no such app: " + appId)
+    }
+  }
+  def securityManager: SecurityManager
+}
+
+private[v1] object UIRootFromServletContext {
+
+  private val attribute = getClass.getCanonicalName
+
+  def setUiRoot(contextHandler: ContextHandler, uiRoot: UIRoot): Unit = {
+    contextHandler.setAttribute(attribute, uiRoot)
+  }
+
+  def getUiRoot(context: ServletContext): UIRoot = {
+    context.getAttribute(attribute).asInstanceOf[UIRoot]
+  }
+}
+
+private[v1] trait UIRootFromServletContext {
+  @Context
+  var servletContext: ServletContext = _
+
+  def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext)
+}
+
+private[v1] class NotFoundException(msg: String) extends WebApplicationException(
+  new NoSuchElementException(msg),
+    Response
+      .status(Response.Status.NOT_FOUND)
+      .entity(ErrorWrapper(msg))
+      .build()
+)
+
+private[v1] class BadParameterException(msg: String) extends WebApplicationException(
+  new IllegalArgumentException(msg),
+  Response
+    .status(Response.Status.BAD_REQUEST)
+    .entity(ErrorWrapper(msg))
+    .build()
+) {
+  def this(param: String, exp: String, actual: String) = {
+    this(raw"""Bad value for parameter "$param".  Expected a $exp, got "$actual"""")
+  }
+}
+
+/**
+ * Signal to JacksonMessageWriter to not convert the message into json (which would result in an
+ * extra set of quotes).
+ */
+private[v1] case class ErrorWrapper(s: String)

http://git-wip-us.apache.org/repos/asf/spark/blob/a70bf06b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala
deleted file mode 100644
index c3ec45f..0000000
--- a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * 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.spark.status.api.v1
-
-import javax.servlet.ServletContext
-import javax.ws.rs._
-import javax.ws.rs.core.{Context, Response}
-
-import com.sun.jersey.api.core.ResourceConfig
-import com.sun.jersey.spi.container.servlet.ServletContainer
-import org.eclipse.jetty.server.handler.ContextHandler
-import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
-
-import org.apache.spark.SecurityManager
-import org.apache.spark.ui.SparkUI
-
-/**
- * Main entry point for serving spark application metrics as json, using JAX-RS.
- *
- * Each resource should have endpoints that return **public** classes defined in api.scala.  Mima
- * binary compatibility checks ensure that we don't inadvertently make changes that break the api.
- * The returned objects are automatically converted to json by jackson with JacksonMessageWriter.
- * In addition, there are a number of tests in HistoryServerSuite that compare the json to "golden
- * files".  Any changes and additions should be reflected there as well -- see the notes in
- * HistoryServerSuite.
- */
-@Path("/v1")
-private[v1] class JsonRootResource extends UIRootFromServletContext {
-
-  @Path("applications")
-  def getApplicationList(): ApplicationListResource = {
-    new ApplicationListResource(uiRoot)
-  }
-
-  @Path("applications/{appId}")
-  def getApplication(): OneApplicationResource = {
-    new OneApplicationResource(uiRoot)
-  }
-
-  @Path("applications/{appId}/{attemptId}/jobs")
-  def getJobs(
-      @PathParam("appId") appId: String,
-      @PathParam("attemptId") attemptId: String): AllJobsResource = {
-    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
-      new AllJobsResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/jobs")
-  def getJobs(@PathParam("appId") appId: String): AllJobsResource = {
-    uiRoot.withSparkUI(appId, None) { ui =>
-      new AllJobsResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/jobs/{jobId: \\d+}")
-  def getJob(@PathParam("appId") appId: String): OneJobResource = {
-    uiRoot.withSparkUI(appId, None) { ui =>
-      new OneJobResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/{attemptId}/jobs/{jobId: \\d+}")
-  def getJob(
-      @PathParam("appId") appId: String,
-      @PathParam("attemptId") attemptId: String): OneJobResource = {
-    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
-      new OneJobResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/executors")
-  def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = {
-    uiRoot.withSparkUI(appId, None) { ui =>
-      new ExecutorListResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/{attemptId}/executors")
-  def getExecutors(
-      @PathParam("appId") appId: String,
-      @PathParam("attemptId") attemptId: String): ExecutorListResource = {
-    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
-      new ExecutorListResource(ui)
-    }
-  }
-
-
-  @Path("applications/{appId}/stages")
-  def getStages(@PathParam("appId") appId: String): AllStagesResource= {
-    uiRoot.withSparkUI(appId, None) { ui =>
-      new AllStagesResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/{attemptId}/stages")
-  def getStages(
-      @PathParam("appId") appId: String,
-      @PathParam("attemptId") attemptId: String): AllStagesResource= {
-    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
-      new AllStagesResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/stages/{stageId: \\d+}")
-  def getStage(@PathParam("appId") appId: String): OneStageResource= {
-    uiRoot.withSparkUI(appId, None) { ui =>
-      new OneStageResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/{attemptId}/stages/{stageId: \\d+}")
-  def getStage(
-      @PathParam("appId") appId: String,
-      @PathParam("attemptId") attemptId: String): OneStageResource = {
-    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
-      new OneStageResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/storage/rdd")
-  def getRdds(@PathParam("appId") appId: String): AllRDDResource = {
-    uiRoot.withSparkUI(appId, None) { ui =>
-      new AllRDDResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/{attemptId}/storage/rdd")
-  def getRdds(
-      @PathParam("appId") appId: String,
-      @PathParam("attemptId") attemptId: String): AllRDDResource = {
-    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
-      new AllRDDResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/storage/rdd/{rddId: \\d+}")
-  def getRdd(@PathParam("appId") appId: String): OneRDDResource = {
-    uiRoot.withSparkUI(appId, None) { ui =>
-      new OneRDDResource(ui)
-    }
-  }
-
-  @Path("applications/{appId}/{attemptId}/storage/rdd/{rddId: \\d+}")
-  def getRdd(
-      @PathParam("appId") appId: String,
-      @PathParam("attemptId") attemptId: String): OneRDDResource = {
-    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
-      new OneRDDResource(ui)
-    }
-  }
-
-}
-
-private[spark] object JsonRootResource {
-
-  def getJsonServlet(uiRoot: UIRoot): ServletContextHandler = {
-    val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS)
-    jerseyContext.setContextPath("/json")
-    val holder:ServletHolder = new ServletHolder(classOf[ServletContainer])
-    holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
-      "com.sun.jersey.api.core.PackagesResourceConfig")
-    holder.setInitParameter("com.sun.jersey.config.property.packages",
-      "org.apache.spark.status.api.v1")
-    holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS,
-      classOf[SecurityFilter].getCanonicalName)
-    UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot)
-    jerseyContext.addServlet(holder, "/*")
-    jerseyContext
-  }
-}
-
-/**
- * This trait is shared by the all the root containers for application UI information --
- * the HistoryServer, the Master UI, and the application UI.  This provides the common
- * interface needed for them all to expose application info as json.
- */
-private[spark] trait UIRoot {
-  def getSparkUI(appKey: String): Option[SparkUI]
-  def getApplicationInfoList: Iterator[ApplicationInfo]
-
-  /**
-   * Get the spark UI with the given appID, and apply a function
-   * to it.  If there is no such app, throw an appropriate exception
-   */
-  def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = {
-    val appKey = attemptId.map(appId + "/" + _).getOrElse(appId)
-    getSparkUI(appKey) match {
-      case Some(ui) =>
-        f(ui)
-      case None => throw new NotFoundException("no such app: " + appId)
-    }
-  }
-  def securityManager: SecurityManager
-}
-
-private[v1] object UIRootFromServletContext {
-
-  private val attribute = getClass.getCanonicalName
-
-  def setUiRoot(contextHandler: ContextHandler, uiRoot: UIRoot): Unit = {
-    contextHandler.setAttribute(attribute, uiRoot)
-  }
-
-  def getUiRoot(context: ServletContext): UIRoot = {
-    context.getAttribute(attribute).asInstanceOf[UIRoot]
-  }
-}
-
-private[v1] trait UIRootFromServletContext {
-  @Context
-  var servletContext: ServletContext = _
-
-  def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext)
-}
-
-private[v1] class NotFoundException(msg: String) extends WebApplicationException(
-  new NoSuchElementException(msg),
-    Response
-      .status(Response.Status.NOT_FOUND)
-      .entity(ErrorWrapper(msg))
-      .build()
-)
-
-private[v1] class BadParameterException(msg: String) extends WebApplicationException(
-  new IllegalArgumentException(msg),
-  Response
-    .status(Response.Status.BAD_REQUEST)
-    .entity(ErrorWrapper(msg))
-    .build()
-) {
-  def this(param: String, exp: String, actual: String) = {
-    this(raw"""Bad value for parameter "$param".  Expected a $exp, got "$actual"""")
-  }
-}
-
-/**
- * Signal to JacksonMessageWriter to not convert the message into json (which would result in an
- * extra set of quotes).
- */
-private[v1] case class ErrorWrapper(s: String)

http://git-wip-us.apache.org/repos/asf/spark/blob/a70bf06b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index bfe4a18..0b11e91 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -19,7 +19,8 @@ package org.apache.spark.ui
 
 import java.util.Date
 
-import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, JsonRootResource, UIRoot}
+import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationAttemptInfo, ApplicationInfo,
+  UIRoot}
 import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext}
 import org.apache.spark.scheduler._
 import org.apache.spark.storage.StorageStatusListener
@@ -64,7 +65,7 @@ private[spark] class SparkUI private (
     attachTab(new ExecutorsTab(this))
     attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
     attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath))
-    attachHandler(JsonRootResource.getJsonServlet(this))
+    attachHandler(ApiRootResource.getServletHandler(this))
     // This should be POST only, but, the YARN AM proxy won't proxy POSTs
     attachHandler(createRedirectHandler(
       "/stages/stage/kill", "/stages", stagesTab.handleKillRequest,

http://git-wip-us.apache.org/repos/asf/spark/blob/a70bf06b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 318ab5d..4adb512 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -198,11 +198,11 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with
   }
 
   def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = {
-    HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/json/v1/$path"))
+    HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/api/v1/$path"))
   }
 
   def getUrl(path: String): String = {
-    HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/json/v1/$path"))
+    HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/api/v1/$path"))
   }
 
   def generateExpectation(name: String, path: String): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/a70bf06b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index 117b2c3..b6f5acc 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -497,7 +497,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       goToUi(sc, "/jobs/job/?id=7")
       find("no-info").get.text should be ("No information to display for job 7")
 
-      val badJob = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get, "jobs/7"))
+      val badJob = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get, "jobs/7"))
       badJob._1 should be (HttpServletResponse.SC_NOT_FOUND)
       badJob._2 should be (None)
       badJob._3 should be (Some("unknown job: 7"))
@@ -540,18 +540,18 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
 
       goToUi(sc, "/stages/stage/?id=12&attempt=0")
       find("no-info").get.text should be ("No information to display for Stage 12 (Attempt 0)")
-      val badStage = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/12/0"))
+      val badStage = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get,"stages/12/0"))
       badStage._1 should be (HttpServletResponse.SC_NOT_FOUND)
       badStage._2 should be (None)
       badStage._3 should be (Some("unknown stage: 12"))
 
-      val badAttempt = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/19/15"))
+      val badAttempt = HistoryServerSuite.getContentAndCode(apiUrl(sc.ui.get,"stages/19/15"))
       badAttempt._1 should be (HttpServletResponse.SC_NOT_FOUND)
       badAttempt._2 should be (None)
       badAttempt._3 should be (Some("unknown attempt for stage 19.  Found attempts: [0]"))
 
       val badStageAttemptList = HistoryServerSuite.getContentAndCode(
-        jsonUrl(sc.ui.get, "stages/12"))
+        apiUrl(sc.ui.get, "stages/12"))
       badStageAttemptList._1 should be (HttpServletResponse.SC_NOT_FOUND)
       badStageAttemptList._2 should be (None)
       badStageAttemptList._3 should be (Some("unknown stage: 12"))
@@ -561,7 +561,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
   test("live UI json application list") {
     withSpark(newSparkContext()) { sc =>
       val appListRawJson = HistoryServerSuite.getUrl(new URL(
-        sc.ui.get.appUIAddress + "/json/v1/applications"))
+        sc.ui.get.appUIAddress + "/api/v1/applications"))
       val appListJsonAst = JsonMethods.parse(appListRawJson)
       appListJsonAst.children.length should be (1)
       val attempts = (appListJsonAst \ "attempts").children
@@ -587,10 +587,10 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
   }
 
   def getJson(ui: SparkUI, path: String): JValue = {
-    JsonMethods.parse(HistoryServerSuite.getUrl(jsonUrl(ui, path)))
+    JsonMethods.parse(HistoryServerSuite.getUrl(apiUrl(ui, path)))
   }
 
-  def jsonUrl(ui: SparkUI, path: String): URL = {
-    new URL(ui.appUIAddress + "/json/v1/applications/test/" + path)
+  def apiUrl(ui: SparkUI, path: String): URL = {
+    new URL(ui.appUIAddress + "/api/v1/applications/test/" + path)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/a70bf06b/docs/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 1e0fc15..e750184 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -178,9 +178,9 @@ Note that the history server only displays completed Spark jobs. One way to sign
 
 In addition to viewing the metrics in the UI, they are also available as JSON.  This gives developers
 an easy way to create new visualizations and monitoring tools for Spark.  The JSON is available for
-both running applications, and in the history server.  The endpoints are mounted at `/json/v1`.  Eg.,
-for the history server, they would typically be accessible at `http://<server-url>:18080/json/v1`, and
-for a running application, at `http://localhost:4040/json/v1`.
+both running applications, and in the history server.  The endpoints are mounted at `/api/v1`.  Eg.,
+for the history server, they would typically be accessible at `http://<server-url>:18080/api/v1`, and
+for a running application, at `http://localhost:4040/api/v1`.
 
 <table class="table">
   <tr><th>Endpoint</th><th>Meaning</th></tr>
@@ -240,12 +240,12 @@ These endpoints have been strongly versioned to make it easier to develop applic
 * Individual fields will never be removed for any given endpoint
 * New endpoints may be added
 * New fields may be added to existing endpoints
-* New versions of the api may be added in the future at a separate endpoint (eg., `json/v2`).  New versions are *not* required to be backwards compatible.
+* New versions of the api may be added in the future at a separate endpoint (eg., `api/v2`).  New versions are *not* required to be backwards compatible.
 * Api versions may be dropped, but only after at least one minor release of co-existing with a new api version
 
 Note that even when examining the UI of a running applications, the `applications/[app-id]` portion is
 still required, though there is only one application available.  Eg. to see the list of jobs for the
-running app, you would go to `http://localhost:4040/json/v1/applications/[app-id]/jobs`.  This is to
+running app, you would go to `http://localhost:4040/api/v1/applications/[app-id]/jobs`.  This is to
 keep the paths consistent in both modes.
 
 # Metrics


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org