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

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


The following commit(s) were added to refs/heads/master by this push:
     new 70f4bbc  [SPARK-28414][WEBUI] UI updates to show resource info in 
Standalone
70f4bbc is described below

commit 70f4bbccc511c60266511c752a1fa37b20be3f8d
Author: wuyi <ngone_5...@163.com>
AuthorDate: Tue Aug 27 08:59:29 2019 -0500

    [SPARK-28414][WEBUI] UI updates to show resource info in Standalone
    
    ## What changes were proposed in this pull request?
    
    Since SPARK-27371 has supported GPU-aware resource scheduling in 
Standalone, this PR adds resources info in Standalone UI.
    
    ## How was this patch tested?
    
    Updated `JsonProtocolSuite` and tested manually.
    
    Master page:
    
    
![masterpage](https://user-images.githubusercontent.com/16397174/62835958-b933c100-bc90-11e9-814f-22bae048303d.png)
    
    Worker page
    
    
![workerpage](https://user-images.githubusercontent.com/16397174/63417947-d2790200-c434-11e9-8979-36b8f558afd3.png)
    
    Application page
    
    
![applicationpage](https://user-images.githubusercontent.com/16397174/62835964-cbadfa80-bc90-11e9-99a2-26e05421619a.png)
    
    Closes #25409 from Ngone51/SPARK-28414.
    
    Authored-by: wuyi <ngone_5...@163.com>
    Signed-off-by: Thomas Graves <tgra...@apache.org>
---
 .../org/apache/spark/deploy/DeployMessage.scala    |  4 +-
 .../org/apache/spark/deploy/JsonProtocol.scala     | 39 +++++++++++-
 .../spark/deploy/StandaloneResourceUtils.scala     | 74 ++++++++++++++++++++++
 .../org/apache/spark/deploy/master/Master.scala    |  4 +-
 .../apache/spark/deploy/master/WorkerInfo.scala    | 29 +++++++--
 .../spark/deploy/master/ui/ApplicationPage.scala   |  9 ++-
 .../apache/spark/deploy/master/ui/MasterPage.scala | 41 ++++++++++--
 .../org/apache/spark/deploy/worker/Worker.scala    | 25 +++++++-
 .../apache/spark/deploy/worker/ui/WorkerPage.scala | 28 +++++++-
 .../apache/spark/resource/ResourceAllocator.scala  |  1 -
 .../spark/resource/ResourceInformation.scala       |  2 +
 .../org/apache/spark/deploy/DeployTestUtils.scala  | 40 ++++++++++--
 .../apache/spark/deploy/JsonProtocolSuite.scala    | 37 +++++++++--
 13 files changed, 298 insertions(+), 35 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala 
b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 3f1d1ae..fba371d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -238,7 +238,9 @@ private[deploy] object DeployMessages {
   case class WorkerStateResponse(host: String, port: Int, workerId: String,
     executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner],
     drivers: List[DriverRunner], finishedDrivers: List[DriverRunner], 
masterUrl: String,
-    cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: 
String) {
+    cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: 
String,
+    resources: Map[String, ResourceInformation] = Map.empty,
+    resourcesUsed: Map[String, ResourceInformation] = Map.empty) {
 
     Utils.checkHost(host)
     assert (port > 0)
diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala 
b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
index 7212696..6c3276c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
@@ -17,15 +17,29 @@
 
 package org.apache.spark.deploy
 
-import org.json4s.JsonAST.JObject
+import org.json4s.JsonAST._
 import org.json4s.JsonDSL._
 
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, 
WorkerStateResponse}
 import org.apache.spark.deploy.master._
 import org.apache.spark.deploy.master.RecoveryState.MasterState
 import org.apache.spark.deploy.worker.ExecutorRunner
+import org.apache.spark.resource.{ResourceInformation, ResourceRequirement}
 
 private[deploy] object JsonProtocol {
+
+  private def writeResourcesInfo(info: Map[String, ResourceInformation]): 
JObject = {
+    val jsonFields = info.map {
+      case (k, v) => JField(k, v.toJson)
+    }
+    JObject(jsonFields.toList)
+  }
+
+  private def writeResourceRequirement(req: ResourceRequirement): JObject = {
+    ("name" -> req.resourceName) ~
+    ("amount" -> req.amount)
+  }
+
   /**
    * Export the [[WorkerInfo]] to a Json object. A [[WorkerInfo]] consists of 
the information of a
    * worker.
@@ -41,6 +55,9 @@ private[deploy] object JsonProtocol {
    *         `memory` total memory of the worker
    *         `memoryused` allocated memory of the worker
    *         `memoryfree` free memory of the worker
+   *         `resources` total resources of the worker
+   *         `resourcesused` allocated resources of the worker
+   *         `resourcesfree` free resources of the worker
    *         `state` state of the worker, see [[WorkerState]]
    *         `lastheartbeat` time in milliseconds that the latest heart beat 
message from the
    *         worker is received
@@ -56,6 +73,9 @@ private[deploy] object JsonProtocol {
     ("memory" -> obj.memory) ~
     ("memoryused" -> obj.memoryUsed) ~
     ("memoryfree" -> obj.memoryFree) ~
+    ("resources" -> writeResourcesInfo(obj.resourcesInfo)) ~
+    ("resourcesused" -> writeResourcesInfo(obj.resourcesInfoUsed)) ~
+    ("resourcesfree" -> writeResourcesInfo(obj.resourcesInfoFree)) ~
     ("state" -> obj.state.toString) ~
     ("lastheartbeat" -> obj.lastHeartbeat)
   }
@@ -71,6 +91,7 @@ private[deploy] object JsonProtocol {
    *         `cores` total cores granted to the application
    *         `user` name of the user who submitted the application
    *         `memoryperslave` minimal memory in MB required to each executor
+   *         `resourcesperslave` minimal resources required to each executor
    *         `submitdate` time in Date that the application is submitted
    *         `state` state of the application, see [[ApplicationState]]
    *         `duration` time in milliseconds that the application has been 
running
@@ -82,6 +103,8 @@ private[deploy] object JsonProtocol {
     ("cores" -> obj.coresGranted) ~
     ("user" -> obj.desc.user) ~
     ("memoryperslave" -> obj.desc.memoryPerExecutorMB) ~
+    ("resourcesperslave" -> obj.desc.resourceReqsPerExecutor
+      .toList.map(writeResourceRequirement)) ~
     ("submitdate" -> obj.submitDate.toString) ~
     ("state" -> obj.state.toString) ~
     ("duration" -> obj.duration)
@@ -95,6 +118,7 @@ private[deploy] object JsonProtocol {
    *         `name` the description of the application
    *         `cores` max cores that can be allocated to the application, 0 
means unlimited
    *         `memoryperslave` minimal memory in MB required to each executor
+   *         `resourcesperslave` minimal resources required to each executor
    *         `user` name of the user who submitted the application
    *         `command` the command string used to submit the application
    */
@@ -102,6 +126,7 @@ private[deploy] object JsonProtocol {
     ("name" -> obj.name) ~
     ("cores" -> obj.maxCores.getOrElse(0)) ~
     ("memoryperslave" -> obj.memoryPerExecutorMB) ~
+    ("resourcesperslave" -> 
obj.resourceReqsPerExecutor.toList.map(writeResourceRequirement)) ~
     ("user" -> obj.user) ~
     ("command" -> obj.command.toString)
   }
@@ -113,6 +138,7 @@ private[deploy] object JsonProtocol {
    * @return a Json object containing the following fields:
    *         `id` an integer identifier of the executor
    *         `memory` memory in MB allocated to the executor
+   *         `resources` resources allocated to the executor
    *         `appid` a string identifier of the application that the executor 
is working on
    *         `appdesc` a Json object of the [[ApplicationDescription]] of the 
application that the
    *         executor is working on
@@ -120,6 +146,7 @@ private[deploy] object JsonProtocol {
   def writeExecutorRunner(obj: ExecutorRunner): JObject = {
     ("id" -> obj.execId) ~
     ("memory" -> obj.memory) ~
+    ("resources" -> writeResourcesInfo(obj.resources)) ~
     ("appid" -> obj.appId) ~
     ("appdesc" -> writeApplicationDescription(obj.appDesc))
   }
@@ -134,6 +161,7 @@ private[deploy] object JsonProtocol {
    *         `state` state of the driver, see [[DriverState]]
    *         `cores` cores allocated to the driver
    *         `memory` memory in MB allocated to the driver
+   *         `resources` resources allocated to the driver
    *         `submitdate` time in Date that the driver is created
    *         `worker` identifier of the worker that the driver is running on
    *         `mainclass` main class of the command string that started the 
driver
@@ -144,6 +172,7 @@ private[deploy] object JsonProtocol {
     ("state" -> obj.state.toString) ~
     ("cores" -> obj.desc.cores) ~
     ("memory" -> obj.desc.mem) ~
+    ("resources" -> writeResourcesInfo(obj.resources)) ~
     ("submitdate" -> obj.submitDate.toString) ~
     ("worker" -> obj.worker.map(_.id).getOrElse("None")) ~
     ("mainclass" -> obj.desc.command.arguments(2))
@@ -162,6 +191,8 @@ private[deploy] object JsonProtocol {
    *         `coresused` cores used by the master
    *         `memory` total memory available of the master
    *         `memoryused` memory used by the master
+   *         `resources` total resources available of the master
+   *         `resourcesused` resources used by the master
    *         `activeapps` a list of Json objects of [[ApplicationInfo]] of the 
active applications
    *         running on the master
    *         `completedapps` a list of Json objects of [[ApplicationInfo]] of 
the applications
@@ -181,6 +212,8 @@ private[deploy] object JsonProtocol {
     ("coresused" -> aliveWorkers.map(_.coresUsed).sum) ~
     ("memory" -> aliveWorkers.map(_.memory).sum) ~
     ("memoryused" -> aliveWorkers.map(_.memoryUsed).sum) ~
+    ("resources" -> 
aliveWorkers.map(_.resourcesInfo).toList.map(writeResourcesInfo)) ~
+    ("resourcesused" -> 
aliveWorkers.map(_.resourcesInfoUsed).toList.map(writeResourcesInfo)) ~
     ("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~
     ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) ~
     ("activedrivers" -> obj.activeDrivers.toList.map(writeDriverInfo)) ~
@@ -200,6 +233,8 @@ private[deploy] object JsonProtocol {
    *         `coreused` used cores of the worker
    *         `memory` total memory of the worker
    *         `memoryused` used memory of the worker
+   *         `resources` total resources of the worker
+   *         `resourcesused` used resources of the worker
    *         `executors` a list of Json objects of [[ExecutorRunner]] of the 
executors running on
    *         the worker
    *         `finishedexecutors` a list of Json objects of [[ExecutorRunner]] 
of the finished
@@ -213,6 +248,8 @@ private[deploy] object JsonProtocol {
     ("coresused" -> obj.coresUsed) ~
     ("memory" -> obj.memory) ~
     ("memoryused" -> obj.memoryUsed) ~
+    ("resources" -> writeResourcesInfo(obj.resources)) ~
+    ("resourcesused" -> writeResourcesInfo(obj.resourcesUsed)) ~
     ("executors" -> obj.executors.map(writeExecutorRunner)) ~
     ("finishedexecutors" -> obj.finishedExecutors.map(writeExecutorRunner))
   }
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala 
b/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala
index b64a36f..d6f9618a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala
@@ -43,6 +43,37 @@ private[spark] object StandaloneResourceUtils extends 
Logging {
   val RESOURCES_LOCK_FILE = "__allocated_resources__.lock"
 
   /**
+   * A mutable resource information which provides more efficient modification 
on addresses.
+   */
+  private[spark] case class MutableResourceInfo(name: String, addresses: 
mutable.HashSet[String]) {
+
+    def + (other: MutableResourceInfo): this.type = {
+      assert(name == other.name, s"Inconsistent resource name, expected $name, 
" +
+        s"but got ${other.name}")
+      other.addresses.foreach(this.addresses.add)
+      this
+    }
+
+    def + (other: ResourceInformation): this.type = {
+      assert(name == other.name, s"Inconsistent resource name, expected $name, 
" +
+        s"but got ${other.name}")
+      other.addresses.foreach(this.addresses.add)
+      this
+    }
+
+    def - (other: ResourceInformation): this.type = {
+      assert(name == other.name, s"Inconsistent resource name, expected $name, 
" +
+        s"but got ${other.name}")
+      other.addresses.foreach(this.addresses.remove)
+      this
+    }
+
+    def toResourceInformation: ResourceInformation = {
+      new ResourceInformation(name, addresses.toArray)
+    }
+  }
+
+  /**
    * Resource allocation used in Standalone only, which tracks assignments with
    * worker/driver(client only) pid.
    */
@@ -345,4 +376,47 @@ private[spark] object StandaloneResourceUtils extends 
Logging {
   def needCoordinate(conf: SparkConf): Boolean = {
     conf.get(SPARK_RESOURCES_COORDINATE)
   }
+
+  def toMutable(immutableResources: Map[String, ResourceInformation])
+    : Map[String, MutableResourceInfo] = {
+    immutableResources.map { case (rName, rInfo) =>
+      val mutableAddress = new mutable.HashSet[String]()
+      mutableAddress ++= rInfo.addresses
+      rName -> MutableResourceInfo(rInfo.name, mutableAddress)
+    }
+  }
+
+  // used for UI
+  def formatResourcesDetails(
+      usedInfo: Map[String, ResourceInformation],
+      freeInfo: Map[String, ResourceInformation]): String = {
+    usedInfo.map { case (rName, rInfo) =>
+      val used = rInfo.addresses.mkString("[", ", ", "]")
+      val free = freeInfo(rName).addresses.mkString("[", ", ", "]")
+      s"$rName: Free: $free / Used: $used"
+    }.mkString(", ")
+  }
+
+  // used for UI
+  def formatResourcesAddresses(resources: Map[String, ResourceInformation]): 
String = {
+    resources.map { case (rName, rInfo) =>
+      s"$rName: ${rInfo.addresses.mkString("[", ", ", "]")}"
+    }.mkString(", ")
+  }
+
+  // used for UI
+  def formatResourcesUsed(
+      resourcesTotal: Map[String, ResourceInformation],
+      resourcesUsed: Map[String, ResourceInformation]): String = {
+    resourcesTotal.map { case (rName, rInfo) =>
+      val used = resourcesUsed(rName).addresses.length
+      val total = rInfo.addresses.length
+      s"$used / $total $rName"
+    }.mkString(", ")
+  }
+
+  // used for UI
+  def formatResourceRequirements(requirements: Seq[ResourceRequirement]): 
String = {
+    requirements.map(req => s"${req.amount} ${req.resourceName}").mkString(", 
")
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 6765519..5588dc8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -644,7 +644,7 @@ private[deploy] class Master(
         val assignedResources = resourceReqsPerExecutor.map {
           req => req.resourceName -> req.amount * assignedExecutorNum
         }.toMap
-        val resourcesFree = usableWorkers(pos).resourcesFree.map {
+        val resourcesFree = usableWorkers(pos).resourcesAmountFree.map {
           case (rName, free) => rName -> (free - 
assignedResources.getOrElse(rName, 0))
         }
         val enoughResources = ResourceUtils.resourcesMeetRequirements(
@@ -752,7 +752,7 @@ private[deploy] class Master(
     val enoughMem = worker.memoryFree >= memoryReq
     val enoughCores = worker.coresFree >= coresReq
     val enoughResources = ResourceUtils.resourcesMeetRequirements(
-      worker.resourcesFree, resourceRequirements)
+      worker.resourcesAmountFree, resourceRequirements)
     enoughMem && enoughCores && enoughResources
   }
 
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index d485db4..9509844 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -18,7 +18,9 @@
 package org.apache.spark.deploy.master
 
 import scala.collection.mutable
+import scala.reflect.ClassTag
 
+import org.apache.spark.deploy.StandaloneResourceUtils.MutableResourceInfo
 import org.apache.spark.resource.{ResourceAllocator, ResourceInformation, 
ResourceRequirement}
 import org.apache.spark.rpc.RpcEndpointRef
 import org.apache.spark.util.Utils
@@ -26,10 +28,6 @@ import org.apache.spark.util.Utils
 private[spark] case class WorkerResourceInfo(name: String, addresses: 
Seq[String])
   extends ResourceAllocator(name, addresses) {
 
-  def toResourceInformation(): ResourceInformation = {
-    new ResourceInformation(name, addresses.toArray)
-  }
-
   def acquire(amount: Int): ResourceInformation = {
     val allocated = availableAddrs.take(amount)
     acquire(allocated)
@@ -45,8 +43,7 @@ private[spark] class WorkerInfo(
     val memory: Int,
     val endpoint: RpcEndpointRef,
     val webUiAddress: String,
-    val resources: Map[String, WorkerResourceInfo],
-    val pid: Int = 0)
+    val resources: Map[String, WorkerResourceInfo])
   extends Serializable {
 
   Utils.checkHost(host)
@@ -64,12 +61,30 @@ private[spark] class WorkerInfo(
 
   def coresFree: Int = cores - coresUsed
   def memoryFree: Int = memory - memoryUsed
-  def resourcesFree: Map[String, Int] = {
+  def resourcesAmountFree: Map[String, Int] = {
     resources.map { case (rName, rInfo) =>
       rName -> rInfo.availableAddrs.length
     }
   }
 
+  def resourcesInfo: Map[String, ResourceInformation] = {
+    resources.map { case (rName, rInfo) =>
+      rName -> new ResourceInformation(rName, rInfo.addresses.toArray)
+    }
+  }
+
+  def resourcesInfoFree: Map[String, ResourceInformation] = {
+    resources.map { case (rName, rInfo) =>
+      rName -> new ResourceInformation(rName, rInfo.availableAddrs.toArray)
+    }
+  }
+
+  def resourcesInfoUsed: Map[String, ResourceInformation] = {
+    resources.map { case (rName, rInfo) =>
+      rName -> new ResourceInformation(rName, rInfo.assignedAddrs.toArray)
+    }
+  }
+
   private def readObject(in: java.io.ObjectInputStream): Unit = 
Utils.tryOrIOException {
     in.defaultReadObject()
     init()
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index bcd7a7e..c7e73bc 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -23,6 +23,7 @@ import scala.xml.Node
 
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, 
RequestMasterState}
 import org.apache.spark.deploy.ExecutorState
+import 
org.apache.spark.deploy.StandaloneResourceUtils.{formatResourceRequirements, 
formatResourcesAddresses}
 import org.apache.spark.deploy.master.ExecutorDesc
 import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage}
 import org.apache.spark.util.Utils
@@ -42,7 +43,8 @@ private[ui] class ApplicationPage(parent: MasterWebUI) 
extends WebUIPage("app")
       return UIUtils.basicSparkPage(request, msg, "Not Found")
     }
 
-    val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", 
"State", "Logs")
+    val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", 
"Resources",
+      "State", "Logs")
     val allExecutors = (app.executors.values ++ 
app.removedExecutors).toSet.toSeq
     // This includes executors that are either still running or have exited 
cleanly
     val executors = allExecutors.filter { exec =>
@@ -83,6 +85,10 @@ private[ui] class ApplicationPage(parent: MasterWebUI) 
extends WebUIPage("app")
               <strong>Executor Memory:</strong>
               {Utils.megabytesToString(app.desc.memoryPerExecutorMB)}
             </li>
+            <li>
+              <strong>Executor Resources:</strong>
+              {formatResourceRequirements(app.desc.resourceReqsPerExecutor)}
+            </li>
             <li><strong>Submit Date:</strong> 
{UIUtils.formatDate(app.submitDate)}</li>
             <li><strong>State:</strong> {app.state}</li>
             {
@@ -139,6 +145,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) 
extends WebUIPage("app")
       </td>
       <td>{executor.cores}</td>
       <td>{executor.memory}</td>
+      <td>{formatResourcesAddresses(executor.resources)}</td>
       <td>{executor.state}</td>
       <td>
         <a 
href={s"$workerUrlRef/logPage?appId=${executor.application.id}&executorId=${executor.
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index 6701465..e8b6145 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -25,7 +25,9 @@ import org.json4s.JValue
 
 import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, 
MasterStateResponse, RequestKillDriver, RequestMasterState}
 import org.apache.spark.deploy.JsonProtocol
+import org.apache.spark.deploy.StandaloneResourceUtils._
 import org.apache.spark.deploy.master._
+import org.apache.spark.resource.ResourceInformation
 import org.apache.spark.ui.{UIUtils, WebUIPage}
 import org.apache.spark.util.Utils
 
@@ -67,24 +69,48 @@ private[ui] class MasterPage(parent: MasterWebUI) extends 
WebUIPage("") {
     }
   }
 
+  private def formatWorkerResourcesDetails(worker: WorkerInfo): String = {
+    val usedInfo = worker.resourcesInfoUsed
+    val freeInfo = worker.resourcesInfoFree
+    formatResourcesDetails(usedInfo, freeInfo)
+  }
+
+  private def formatMasterResourcesInUse(aliveWorkers: Array[WorkerInfo]): 
String = {
+    val totalInfo = aliveWorkers.map(_.resourcesInfo)
+      .map(resources => toMutable(resources))
+      .flatMap(_.toIterator)
+      .groupBy(_._1) // group by resource name
+      .map { case (rName, rInfoArr) =>
+        rName -> rInfoArr.map(_._2).reduce(_ + _)
+      }.map { case (k, v) => (k, v.toResourceInformation) }
+    val usedInfo = aliveWorkers.map(_.resourcesInfoUsed)
+      .map (resources => toMutable(resources))
+      .flatMap(_.toIterator)
+      .groupBy(_._1) // group by resource name
+      .map { case (rName, rInfoArr) =>
+      rName -> rInfoArr.map(_._2).reduce(_ + _)
+    }.map { case (k, v) => (k, v.toResourceInformation) }
+    formatResourcesUsed(totalInfo, usedInfo)
+  }
+
   /** Index view listing applications and executors */
   def render(request: HttpServletRequest): Seq[Node] = {
     val state = getMasterState
 
-    val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", "Memory")
+    val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", 
"Memory", "Resources")
     val workers = state.workers.sortBy(_.id)
     val aliveWorkers = state.workers.filter(_.state == WorkerState.ALIVE)
     val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers)
 
-    val appHeaders = Seq("Application ID", "Name", "Cores", "Memory per 
Executor", "Submitted Time",
-      "User", "State", "Duration")
+    val appHeaders = Seq("Application ID", "Name", "Cores", "Memory per 
Executor",
+      "Resources Per Executor", "Submitted Time", "User", "State", "Duration")
     val activeApps = state.activeApps.sortBy(_.startTime).reverse
     val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps)
     val completedApps = state.completedApps.sortBy(_.endTime).reverse
     val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, 
completedApps)
 
     val driverHeaders = Seq("Submission ID", "Submitted Time", "Worker", 
"State", "Cores",
-      "Memory", "Main Class")
+      "Memory", "Resources", "Main Class")
     val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse
     val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, 
activeDrivers)
     val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
@@ -113,6 +139,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends 
WebUIPage("") {
               <li><strong>Memory in use:</strong>
                 {Utils.megabytesToString(aliveWorkers.map(_.memory).sum)} 
Total,
                 {Utils.megabytesToString(aliveWorkers.map(_.memoryUsed).sum)} 
Used</li>
+              <li><strong>Resources in use:</strong>
+                {formatMasterResourcesInUse(aliveWorkers)}</li>
               <li><strong>Applications:</strong>
                 {state.activeApps.length} <a href="#running-app">Running</a>,
                 {state.completedApps.length} <a 
href="#completed-app">Completed</a> </li>
@@ -236,6 +264,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends 
WebUIPage("") {
         {Utils.megabytesToString(worker.memory)}
         ({Utils.megabytesToString(worker.memoryUsed)} Used)
       </td>
+      <td>{formatWorkerResourcesDetails(worker)}</td>
     </tr>
   }
 
@@ -272,6 +301,9 @@ private[ui] class MasterPage(parent: MasterWebUI) extends 
WebUIPage("") {
       <td sorttable_customkey={app.desc.memoryPerExecutorMB.toString}>
         {Utils.megabytesToString(app.desc.memoryPerExecutorMB)}
       </td>
+      <td>
+        {formatResourceRequirements(app.desc.resourceReqsPerExecutor)}
+      </td>
       <td>{UIUtils.formatDate(app.submitDate)}</td>
       <td>{app.desc.user}</td>
       <td>{app.state.toString}</td>
@@ -312,6 +344,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends 
WebUIPage("") {
       <td sorttable_customkey={driver.desc.mem.toString}>
         {Utils.megabytesToString(driver.desc.mem.toLong)}
       </td>
+      <td>{formatResourcesAddresses(driver.resources)}</td>
       <td>{driver.desc.command.arguments(2)}</td>
     </tr>
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala 
b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 899593d..3731b6a 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -34,7 +34,7 @@ import org.apache.spark.deploy.{Command, ExecutorDescription, 
ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.ExternalShuffleService
 import org.apache.spark.deploy.StandaloneResourceUtils._
-import org.apache.spark.deploy.master.{DriverState, Master}
+import org.apache.spark.deploy.master.{DriverState, Master, WorkerResourceInfo}
 import org.apache.spark.deploy.worker.ui.WorkerWebUI
 import org.apache.spark.internal.{config, Logging}
 import org.apache.spark.internal.config.Tests.IS_TESTING
@@ -185,6 +185,7 @@ private[deploy] class Worker(
 
   var coresUsed = 0
   var memoryUsed = 0
+  val resourcesUsed = new HashMap[String, MutableResourceInfo]()
 
   def coresFree: Int = cores - coresUsed
   def memoryFree: Int = memory - memoryUsed
@@ -242,6 +243,21 @@ private[deploy] class Worker(
           System.exit(1)
         }
     }
+    resources.keys.foreach { rName =>
+      resourcesUsed(rName) = MutableResourceInfo(rName, new HashSet[String])
+    }
+  }
+
+  private def addResourcesUsed(deltaInfo: Map[String, ResourceInformation]): 
Unit = {
+    deltaInfo.foreach { case (rName, rInfo) =>
+      resourcesUsed(rName) = resourcesUsed(rName) + rInfo
+    }
+  }
+
+  private def removeResourcesUsed(deltaInfo: Map[String, 
ResourceInformation]): Unit = {
+    deltaInfo.foreach { case (rName, rInfo) =>
+      resourcesUsed(rName) = resourcesUsed(rName) - rInfo
+    }
   }
 
   /**
@@ -588,6 +604,7 @@ private[deploy] class Worker(
           manager.start()
           coresUsed += cores_
           memoryUsed += memory_
+          addResourcesUsed(resources_)
         } catch {
           case e: Exception =>
             logError(s"Failed to launch executor $appId/$execId for 
${appDesc.name}.", e)
@@ -634,6 +651,7 @@ private[deploy] class Worker(
 
       coresUsed += driverDesc.cores
       memoryUsed += driverDesc.mem
+      addResourcesUsed(resources_)
 
     case KillDriver(driverId) =>
       logInfo(s"Asked to kill driver $driverId")
@@ -660,7 +678,8 @@ private[deploy] class Worker(
       context.reply(WorkerStateResponse(host, port, workerId, 
executors.values.toList,
         finishedExecutors.values.toList, drivers.values.toList,
         finishedDrivers.values.toList, activeMasterUrl, cores, memory,
-        coresUsed, memoryUsed, activeMasterWebUiUrl))
+        coresUsed, memoryUsed, activeMasterWebUiUrl, resources,
+        resourcesUsed.toMap.map { case (k, v) => (k, 
v.toResourceInformation)}))
   }
 
   override def onDisconnected(remoteAddress: RpcAddress): Unit = {
@@ -773,6 +792,7 @@ private[deploy] class Worker(
     trimFinishedDriversIfNecessary()
     memoryUsed -= driver.driverDesc.mem
     coresUsed -= driver.driverDesc.cores
+    removeResourcesUsed(driver.resources)
   }
 
   private[worker] def handleExecutorStateChanged(executorStateChanged: 
ExecutorStateChanged):
@@ -794,6 +814,7 @@ private[deploy] class Worker(
           trimFinishedExecutorsIfNecessary()
           coresUsed -= executor.cores
           memoryUsed -= executor.memory
+          removeResourcesUsed(executor.resources)
 
           if (CLEANUP_FILES_AFTER_EXECUTOR_EXIT) {
             
shuffleService.executorRemoved(executorStateChanged.execId.toString, appId)
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala 
b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala
index aa4e28d..1a4ba06 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala
@@ -25,8 +25,10 @@ import org.json4s.JValue
 
 import org.apache.spark.deploy.{ExecutorState, JsonProtocol}
 import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, 
WorkerStateResponse}
+import 
org.apache.spark.deploy.StandaloneResourceUtils.{formatResourcesAddresses, 
formatResourcesDetails}
 import org.apache.spark.deploy.master.DriverState
 import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
+import org.apache.spark.resource.ResourceInformation
 import org.apache.spark.ui.{UIUtils, WebUIPage}
 import org.apache.spark.util.Utils
 
@@ -38,10 +40,25 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends 
WebUIPage("") {
     JsonProtocol.writeWorkerState(workerState)
   }
 
+  private def formatWorkerResourcesDetails(workerState: WorkerStateResponse): 
String = {
+    val totalInfo = workerState.resources
+    val usedInfo = workerState.resourcesUsed
+    val freeInfo = totalInfo.map { case (rName, rInfo) =>
+      val freeAddresses = if (usedInfo.contains(rName)) {
+        rInfo.addresses.diff(usedInfo(rName).addresses)
+      } else {
+        rInfo.addresses
+      }
+      rName -> new ResourceInformation(rName, freeAddresses)
+    }
+    formatResourcesDetails(usedInfo, freeInfo)
+  }
+
   def render(request: HttpServletRequest): Seq[Node] = {
     val workerState = 
workerEndpoint.askSync[WorkerStateResponse](RequestWorkerState)
 
-    val executorHeaders = Seq("ExecutorID", "Cores", "State", "Memory", "Job 
Details", "Logs")
+    val executorHeaders = Seq("ExecutorID", "State", "Cores", "Memory", 
"Resources",
+      "Job Details", "Logs")
     val runningExecutors = workerState.executors
     val runningExecutorTable =
       UIUtils.listingTable(executorHeaders, executorRow, runningExecutors)
@@ -49,7 +66,8 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends 
WebUIPage("") {
     val finishedExecutorTable =
       UIUtils.listingTable(executorHeaders, executorRow, finishedExecutors)
 
-    val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", 
"Memory", "Logs", "Notes")
+    val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", 
"Memory", "Resources",
+      "Logs", "Notes")
     val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse
     val runningDriverTable = UIUtils.listingTable[DriverRunner](driverHeaders,
       driverRow(workerState.workerId, _), runningDrivers)
@@ -71,6 +89,8 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends 
WebUIPage("") {
             <li><strong>Cores:</strong> {workerState.cores} 
({workerState.coresUsed} Used)</li>
             <li><strong>Memory:</strong> 
{Utils.megabytesToString(workerState.memory)}
               ({Utils.megabytesToString(workerState.memoryUsed)} Used)</li>
+            <li><strong>Resources:</strong>
+              {formatWorkerResourcesDetails(workerState)}</li>
           </ul>
           <p><a href={workerState.masterWebUiUrl}>Back to Master</a></p>
         </div>
@@ -147,11 +167,12 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends 
WebUIPage("") {
 
     <tr>
       <td>{executor.execId}</td>
-      <td>{executor.cores}</td>
       <td>{executor.state}</td>
+      <td>{executor.cores}</td>
       <td sorttable_customkey={executor.memory.toString}>
         {Utils.megabytesToString(executor.memory)}
       </td>
+      <td>{formatResourcesAddresses(executor.resources)}</td>
       <td>
         <ul class="unstyled">
           <li><strong>ID:</strong> {executor.appId}</li>
@@ -189,6 +210,7 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends 
WebUIPage("") {
       <td sorttable_customkey={driver.driverDesc.mem.toString}>
         {Utils.megabytesToString(driver.driverDesc.mem)}
       </td>
+      <td>{formatResourcesAddresses(driver.resources)}</td>
       <td>
         <a 
href={s"$workerUrlRef/logPage?driverId=${driver.driverId}&logType=stdout"}>stdout</a>
         <a 
href={s"$workerUrlRef/logPage?driverId=${driver.driverId}&logType=stderr"}>stderr</a>
diff --git 
a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala 
b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala
index 719f34db..6f9b9c0 100644
--- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala
+++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala
@@ -45,7 +45,6 @@ class ResourceAllocator(name: String, addresses: Seq[String]) 
extends Serializab
 
   /**
    * Sequence of currently assigned resource addresses.
-   * Exposed for testing only.
    */
   private[spark] def assignedAddrs: Seq[String] = addressAvailabilityMap
     .flatMap { case (addr, available) =>
diff --git 
a/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala 
b/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala
index 96aef74..8b8bd7b 100644
--- a/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala
+++ b/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala
@@ -52,6 +52,8 @@ class ResourceInformation(
   }
 
   override def hashCode(): Int = Seq(name, addresses.toSeq).hashCode()
+
+  def toJson(): JValue = ResourceInformationJson(name, addresses).toJValue
 }
 
 private[spark] object ResourceInformation {
diff --git a/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala 
b/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala
index a2c4669..31f065e 100644
--- a/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala
@@ -20,8 +20,10 @@ package org.apache.spark.deploy
 import java.io.File
 
 import org.apache.spark.{SecurityManager, SparkConf}
-import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
+import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, 
WorkerInfo, WorkerResourceInfo}
 import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
+import org.apache.spark.resource.{ResourceInformation, ResourceRequirement}
+import org.apache.spark.resource.ResourceUtils.{FPGA, GPU}
 
 private[deploy] object DeployTestUtils {
   def createAppDesc(): ApplicationDescription = {
@@ -32,7 +34,8 @@ private[deploy] object DeployTestUtils {
   def createAppInfo() : ApplicationInfo = {
     val appDesc = createAppDesc()
     val appInfo = new ApplicationInfo(JsonConstants.appInfoStartTime,
-      "id", appDesc, JsonConstants.submitDate, null, Int.MaxValue)
+      "id", appDesc.copy(resourceReqsPerExecutor = createResourceRequirement),
+      JsonConstants.submitDate, null, Int.MaxValue)
     appInfo.endTime = JsonConstants.currTimeInMillis
     appInfo
   }
@@ -45,17 +48,29 @@ private[deploy] object DeployTestUtils {
   def createDriverDesc(): DriverDescription =
     new DriverDescription("hdfs://some-dir/some.jar", 100, 3, false, 
createDriverCommand())
 
-  def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3",
-    createDriverDesc(), JsonConstants.submitDate)
+  def createDriverInfo(): DriverInfo = {
+    val dDesc = createDriverDesc().copy(resourceReqs = 
createResourceRequirement)
+    val dInfo = new DriverInfo(3, "driver-3", dDesc, JsonConstants.submitDate)
+    dInfo.withResources(createResourceInformation)
+    dInfo
+  }
 
   def createWorkerInfo(): WorkerInfo = {
+    val gpuResource = new WorkerResourceInfo(GPU, Seq("0", "1", "2"))
+    val fpgaResource = new WorkerResourceInfo(FPGA, Seq("3", "4", "5"))
+    val resources = Map(GPU -> gpuResource, FPGA -> fpgaResource)
     val workerInfo = new WorkerInfo("id", "host", 8080, 4, 1234, null,
-      "http://publicAddress:80";, Map.empty)
+      "http://publicAddress:80";, resources)
     workerInfo.lastHeartbeat = JsonConstants.currTimeInMillis
     workerInfo
   }
 
-  def createExecutorRunner(execId: Int): ExecutorRunner = {
+  def createExecutorRunner(execId: Int, withResources: Boolean = false): 
ExecutorRunner = {
+    val resources = if (withResources) {
+      createResourceInformation
+    } else {
+      Map.empty[String, ResourceInformation]
+    }
     new ExecutorRunner(
       "appId",
       execId,
@@ -73,7 +88,8 @@ private[deploy] object DeployTestUtils {
       "spark://worker",
       new SparkConf,
       Seq("localDir"),
-      ExecutorState.RUNNING)
+      ExecutorState.RUNNING,
+      resources)
   }
 
   def createDriverRunner(driverId: String): DriverRunner = {
@@ -88,4 +104,14 @@ private[deploy] object DeployTestUtils {
       "spark://worker",
       new SecurityManager(conf))
   }
+
+  private def createResourceInformation: Map[String, ResourceInformation] = {
+    val gpuResource = new ResourceInformation(GPU, Array("0", "1", "2"))
+    val fpgaResource = new ResourceInformation(FPGA, Array("3", "4", "5"))
+    Map(GPU -> gpuResource, FPGA -> fpgaResource)
+  }
+
+  private def createResourceRequirement: Seq[ResourceRequirement] = {
+    Seq(ResourceRequirement("gpu", 3), ResourceRequirement("fpga", 3))
+  }
 }
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala 
b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 1903130..ad402c0 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -51,7 +51,7 @@ class JsonProtocolSuite extends SparkFunSuite with 
JsonTestUtils {
   }
 
   test("writeExecutorRunner") {
-    val output = JsonProtocol.writeExecutorRunner(createExecutorRunner(123))
+    val output = JsonProtocol.writeExecutorRunner(createExecutorRunner(123, 
true))
     assertValidJson(output)
     assertValidDataInJson(output, 
JsonMethods.parse(JsonConstants.executorRunnerJsonStr))
   }
@@ -78,8 +78,8 @@ class JsonProtocolSuite extends SparkFunSuite with 
JsonTestUtils {
 
   test("writeWorkerState") {
     val executors = List[ExecutorRunner]()
-    val finishedExecutors = List[ExecutorRunner](createExecutorRunner(123),
-      createExecutorRunner(123))
+    val finishedExecutors = List[ExecutorRunner](createExecutorRunner(123, 
true),
+      createExecutorRunner(123, true))
     val drivers = List(createDriverRunner("driverId"))
     val finishedDrivers = List(createDriverRunner("driverId"), 
createDriverRunner("driverId"))
     val stateResponse = new WorkerStateResponse("host", 8080, "workerId", 
executors,
@@ -106,7 +106,10 @@ object JsonConstants {
     """
       |{"id":"id","starttime":3,"name":"name",
       |"cores":0,"user":"%s",
-      |"memoryperslave":1234,"submitdate":"%s",
+      |"memoryperslave":1234,
+      |"resourcesperslave":[{"name":"gpu",
+      |"amount":3},{"name":"fpga","amount":3}],
+      |"submitdate":"%s",
       |"state":"WAITING","duration":%d}
     """.format(System.getProperty("user.name", "<unknown>"),
         submitDate.toString, currTimeInMillis - appInfoStartTime).stripMargin
@@ -117,18 +120,28 @@ object JsonConstants {
       |"webuiaddress":"http://publicAddress:80";,
       |"cores":4,"coresused":0,"coresfree":4,
       |"memory":1234,"memoryused":0,"memoryfree":1234,
+      |"resources":{"gpu":{"name":"gpu","addresses":
+      |["0","1","2"]},"fpga":{"name":"fpga","addresses"
+      |:["3","4","5"]}},"resourcesused":{"gpu":
+      |{"name":"gpu","addresses":[]},"fpga":
+      |{"name":"fpga","addresses":[]}},"resourcesfree":
+      |{"gpu":{"name":"gpu","addresses":["2","1","0"]},
+      |"fpga":{"name":"fpga","addresses":["5","4","3"]}},
       |"state":"ALIVE","lastheartbeat":%d}
     """.format(currTimeInMillis).stripMargin
 
   val appDescJsonStr =
     """
-      |{"name":"name","cores":4,"memoryperslave":1234,
+      |{"name":"name","cores":4,"memoryperslave":1234,"resourcesperslave":[],
       |"user":"%s","command":"Command(mainClass,List(arg1, 
arg2),Map(),List(),List(),List())"}
     """.format(System.getProperty("user.name", "<unknown>")).stripMargin
 
   val executorRunnerJsonStr =
     """
-      |{"id":123,"memory":1234,"appid":"appId",
+      |{"id":123,"memory":1234,"resources":
+      |{"gpu":{"name":"gpu","addresses":["0","1","2"]},
+      |"fpga":{"name":"fpga","addresses":["3","4","5"]}},
+      |"appid":"appId",
       |"appdesc":%s}
     """.format(appDescJsonStr).stripMargin
 
@@ -136,6 +149,9 @@ object JsonConstants {
     """
       |{"id":"driver-3","starttime":"3",
       |"state":"SUBMITTED","cores":3,"memory":100,
+      |"resources":{"gpu":{"name":"gpu","addresses":
+      |["0","1","2"]},"fpga":{"name":"fpga",
+      |"addresses":["3","4","5"]}},
       |"submitdate":"%s","worker":"None",
       |"mainclass":"mainClass"}
     """.format(submitDate.toString).stripMargin
@@ -146,6 +162,14 @@ object JsonConstants {
       |"workers":[%s,%s],
       |"aliveworkers":2,
       |"cores":8,"coresused":0,"memory":2468,"memoryused":0,
+      |"resources":[{"gpu":{"name":"gpu","addresses":
+      |["0","1","2"]},"fpga":{"name":"fpga","addresses":
+      |["3","4","5"]}},{"gpu":{"name":"gpu","addresses":
+      |["0","1","2"]},"fpga":{"name":"fpga","addresses":
+      |["3","4","5"]}}],"resourcesused":[{"gpu":{"name":
+      |"gpu","addresses":[]},"fpga":{"name":"fpga","addresses":[]}}
+      |,{"gpu":{"name":"gpu","addresses":[]},"fpga":
+      |{"name":"fpga","addresses":[]}}],
       |"activeapps":[%s],"completedapps":[],
       |"activedrivers":[%s],
       |"completeddrivers":[%s],
@@ -158,6 +182,7 @@ object JsonConstants {
       |{"id":"workerId","masterurl":"masterUrl",
       |"masterwebuiurl":"masterWebUiUrl",
       |"cores":4,"coresused":4,"memory":1234,"memoryused":1234,
+      |"resources":{},"resourcesused":{},
       |"executors":[],
       |"finishedexecutors":[%s,%s]}
     """.format(executorRunnerJsonStr, executorRunnerJsonStr).stripMargin


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

Reply via email to