[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-27 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29190417
  
--- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.Date
+
+import scala.collection.Map
+
+import org.apache.spark.JobExecutionStatus
+
+class ApplicationInfo(
--- End diff --

this was intentional, to get them covered by mima.  I also think the goal 
is provide more stability than implied by `@DeveloperApi`


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-27 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29183894
  
--- Diff: 
core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java ---
@@ -0,0 +1,45 @@
+package org.apache.spark.status.api.v1;/*
+ * 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.
+ */
+
+import org.apache.spark.status.api.EnumUtil;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public enum TaskSorting {
--- End diff --

unfortunately jersey requires them to be public -- I'll tag w/ 
`@DeveloperApi`


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089887
  
--- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.Date
+
+import scala.collection.Map
+
+import org.apache.spark.JobExecutionStatus
+
+class ApplicationInfo(
+  val id: String,
+  val name: String,
+  val startTime: Date,
+  val endTime: Date,
+  val sparkUser: String,
+  val completed: Boolean = false
+)
+
+class ExecutorStageSummary(
+  val taskTime : Long,
+  val failedTasks : Int,
+  val succeededTasks : Int,
+  val inputBytes : Long,
+  val outputBytes : Long,
+  val shuffleRead : Long,
+  val shuffleWrite : Long,
+  val memoryBytesSpilled : Long,
+  val diskBytesSpilled : Long
+)
+
+class ExecutorSummary(
+  val id: String,
+  val hostPort: String,
+  val rddBlocks: Int,
+  val memoryUsed: Long,
+  val diskUsed: Long,
+  val activeTasks: Int,
+  val failedTasks: Int,
+  val completedTasks: Int,
+  val totalTasks: Int,
+  val totalDuration: Long,
+  val totalInputBytes: Long,
+  val totalShuffleRead: Long,
+  val totalShuffleWrite: Long,
+  val maxMemory: Long,
+  val executorLogs: Map[String, String]
+)
+
+class JobData(
+  val jobId: Int,
+  val name: String,
+  val description: Option[String],
+  val submissionTime: Option[Date],
+  val completionTime: Option[Date],
+  val stageIds: Seq[Int],
+  val jobGroup: Option[String],
+  val status: JobExecutionStatus,
+  val numTasks: Int,
+  val numActiveTasks: Int,
+  val numCompletedTasks: Int,
+  val numSkippedTasks: Int,
+  val numFailedTasks: Int,
+  val numActiveStages: Int,
+  val numCompletedStages: Int,
+  val numSkippedStages: Int,
+  val numFailedStages: Int
+)
+
+// Q: should Tachyon size go in here as well?  currently the UI only shows 
it on the overall storage
+// page ... does anybody pay attention to it?
+class RDDStorageInfo(
+  val id: Int,
+  val name: String,
+  val numPartitions: Int,
+  val numCachedPartitions: Int,
+  val storageLevel: String,
+  val memoryUsed: Long,
+  val diskUsed: Long,
+  val dataDistribution: Option[Seq[RDDDataDistribution]],
+  val partitions: Option[Seq[RDDPartitionInfo]]
+)
+
+class RDDDataDistribution(
+  val address: String,
+  val memoryUsed: Long,
+  val memoryRemaining: Long,
+  val diskUsed: Long
+)
+
+class RDDPartitionInfo(
+  val blockName: String,
+  val storageLevel: String,
+  val memoryUsed: Long,
+  val diskUsed: Long,
+  val executors: Seq[String]
+)
+
+class StageData(
+  val status: StageStatus,
+  val stageId: Int,
+  val attemptId: Int,
+  val numActiveTasks: Int ,
+  val numCompleteTasks: Int,
+  val numFailedTasks: Int,
+
+  val executorRunTime: Long,
+
+  val inputBytes: Long,
+  val inputRecords: Long,
+  val outputBytes: Long,
+  val outputRecords: Long,
+  val shuffleReadBytes: Long,
+  val shuffleReadRecords: Long,
+  val shuffleWriteBytes: Long,
+  val shuffleWriteRecords: Long,
+  val memoryBytesSpilled: Long,
+  val diskBytesSpilled: Long,
+
+  val name: String,
+  val details: String,
+  val schedulingPool: String,
+
+  val accumulatorUpdates: Seq[AccumulableInfo],
+  val tasks: Option[Map[Long, TaskData]],
+  val executorSummary:Option[Map[String,ExecutorStageSummary]]
+)
+
+class TaskData(
+  val taskId: Long,
+  val index: Int,
+  val attempt: Int,
+  val launchTime: Date,
+  val executorId: String,
+  val host: String,
+  val taskLocality: String,
+  val speculative: Boolean,
+  val accumulatorUpdates: Seq[AccumulableInfo],
+  val 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089928
  
--- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.Date
+
+import scala.collection.Map
+
+import org.apache.spark.JobExecutionStatus
+
+class ApplicationInfo(
+  val id: String,
+  val name: String,
+  val startTime: Date,
+  val endTime: Date,
+  val sparkUser: String,
+  val completed: Boolean = false
+)
+
+class ExecutorStageSummary(
+  val taskTime : Long,
+  val failedTasks : Int,
+  val succeededTasks : Int,
+  val inputBytes : Long,
+  val outputBytes : Long,
+  val shuffleRead : Long,
+  val shuffleWrite : Long,
+  val memoryBytesSpilled : Long,
+  val diskBytesSpilled : Long
+)
+
+class ExecutorSummary(
+  val id: String,
+  val hostPort: String,
+  val rddBlocks: Int,
+  val memoryUsed: Long,
+  val diskUsed: Long,
+  val activeTasks: Int,
+  val failedTasks: Int,
+  val completedTasks: Int,
+  val totalTasks: Int,
+  val totalDuration: Long,
+  val totalInputBytes: Long,
+  val totalShuffleRead: Long,
+  val totalShuffleWrite: Long,
+  val maxMemory: Long,
+  val executorLogs: Map[String, String]
+)
+
+class JobData(
+  val jobId: Int,
+  val name: String,
+  val description: Option[String],
+  val submissionTime: Option[Date],
+  val completionTime: Option[Date],
+  val stageIds: Seq[Int],
+  val jobGroup: Option[String],
+  val status: JobExecutionStatus,
+  val numTasks: Int,
+  val numActiveTasks: Int,
+  val numCompletedTasks: Int,
+  val numSkippedTasks: Int,
+  val numFailedTasks: Int,
+  val numActiveStages: Int,
+  val numCompletedStages: Int,
+  val numSkippedStages: Int,
+  val numFailedStages: Int
+)
+
+// Q: should Tachyon size go in here as well?  currently the UI only shows 
it on the overall storage
+// page ... does anybody pay attention to it?
+class RDDStorageInfo(
+  val id: Int,
+  val name: String,
+  val numPartitions: Int,
+  val numCachedPartitions: Int,
+  val storageLevel: String,
+  val memoryUsed: Long,
+  val diskUsed: Long,
+  val dataDistribution: Option[Seq[RDDDataDistribution]],
+  val partitions: Option[Seq[RDDPartitionInfo]]
+)
+
+class RDDDataDistribution(
+  val address: String,
+  val memoryUsed: Long,
+  val memoryRemaining: Long,
+  val diskUsed: Long
+)
+
+class RDDPartitionInfo(
+  val blockName: String,
+  val storageLevel: String,
+  val memoryUsed: Long,
+  val diskUsed: Long,
+  val executors: Seq[String]
+)
+
+class StageData(
+  val status: StageStatus,
+  val stageId: Int,
+  val attemptId: Int,
+  val numActiveTasks: Int ,
+  val numCompleteTasks: Int,
+  val numFailedTasks: Int,
+
+  val executorRunTime: Long,
+
+  val inputBytes: Long,
+  val inputRecords: Long,
+  val outputBytes: Long,
+  val outputRecords: Long,
+  val shuffleReadBytes: Long,
+  val shuffleReadRecords: Long,
+  val shuffleWriteBytes: Long,
+  val shuffleWriteRecords: Long,
+  val memoryBytesSpilled: Long,
+  val diskBytesSpilled: Long,
+
+  val name: String,
+  val details: String,
+  val schedulingPool: String,
+
+  val accumulatorUpdates: Seq[AccumulableInfo],
+  val tasks: Option[Map[Long, TaskData]],
+  val executorSummary:Option[Map[String,ExecutorStageSummary]]
+)
+
+class TaskData(
+  val taskId: Long,
+  val index: Int,
+  val attempt: Int,
+  val launchTime: Date,
+  val executorId: String,
+  val host: String,
+  val taskLocality: String,
+  val speculative: Boolean,
+  val accumulatorUpdates: Seq[AccumulableInfo],
+  val 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089953
  
--- Diff: core/src/main/scala/org/apache/spark/ui/SparkUI.scala ---
@@ -17,6 +17,10 @@
 
 package org.apache.spark.ui
 
+import java.util.Date
+
+import org.apache.spark.status.api.v1.{ApplicationInfo, JsonRootResource}
+import org.apache.spark.status.api.v1.UIRoot
--- End diff --

Can merge with previous import.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29090285
  
--- Diff: 
core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala ---
@@ -14,22 +14,170 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.spark.deploy.history
 
-import javax.servlet.http.HttpServletRequest
-
-import scala.collection.mutable
+import java.io.{File, FileInputStream, FileWriter, IOException}
+import java.net.{HttpURLConnection, URL}
+import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
 
-import org.apache.hadoop.fs.Path
-import org.mockito.Mockito.{when}
-import org.scalatest.FunSuite
-import org.scalatest.Matchers
+import org.apache.commons.io.{FileUtils, IOUtils}
+import org.mockito.Mockito.when
+import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
 import org.scalatest.mock.MockitoSugar
 
+import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf}
 import org.apache.spark.ui.SparkUI
 
-class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar {
+class HistoryServerSuite extends FunSuite with BeforeAndAfter with 
Matchers with MockitoSugar
+  with JsonTestUtils {
+
+  private val logDir = new File(src/test/resources/spark-events)
+  private val expRoot = new 
File(src/test/resources/HistoryServerExpectations/)
+
+  private var provider: FsHistoryProvider = null
+  private var server: HistoryServer = null
+  private var port: Int = -1
+
+  def init(): Unit = {
+val conf = new SparkConf()
+  .set(spark.history.fs.logDirectory, logDir.getAbsolutePath)
+  .set(spark.history.fs.updateInterval, 0)
+  .set(spark.testing, true)
+provider = new FsHistoryProvider(conf)
+provider.checkForLogs()
+val securityManager = new SecurityManager(conf)
+
+server = new HistoryServer(conf, provider, securityManager, 18080)
+server.initialize()
+server.bind()
+port = server.boundPort
+  }
+  def stop(): Unit = {
+server.stop()
+  }
+
+  before {
+init()
+  }
+
+  after{
+stop()
+  }
+
+  val cases = Seq(
+application list json - applications,
+completed app list json - applications?status=completed,
+running app list json - applications?status=running,
+minDate app list json - applications?minDate=2015-02-10,
+maxDate app list json - applications?maxDate=2015-02-10,
+maxDate2 app list json - 
applications?maxDate=2015-02-03T10:42:40.000CST,
+one app json - applications/local-1422981780767,
+job list json - applications/local-1422981780767/jobs,
+one job json - applications/local-1422981780767/jobs/0,
+succeeded job list json - 
applications/local-1422981780767/jobs?status=succeeded,
+succeededfailed job list json -
+  
applications/local-1422981780767/jobs?status=succeededstatus=failed,
+executor list json - applications/local-1422981780767/executors,
+stage list json - applications/local-1422981780767/stages,
+complete stage list json - 
applications/local-1422981780767/stages?status=complete,
+failed stage list json - 
applications/local-1422981780767/stages?status=failed,
+one stage json - applications/local-1422981780767/stages/1,
+one stage attempt json - 
applications/local-1422981780767/stages/1/0,
+
+stage task summary - 
applications/local-1427397477963/stages/20/0/taskSummary,
+stage task summary w/ custom quantiles -
+  
applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99,
+
+stage task list - 
applications/local-1427397477963/stages/20/0/taskList,
+stage task list w/ offset  length -
+  
applications/local-1427397477963/stages/20/0/taskList?offset=10length=50,
+stage task list w/ sortBy -
+  
applications/local-1427397477963/stages/20/0/taskList?sortBy=DecreasingRuntime,
+stage task list w/ sortBy short names: -runtime -
+  
applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime,
+stage task list w/ sortBy short names: runtime -
+  
applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime,
+
+stage list with accumulable json - 
applications/local-1426533911241/stages,
+stage with accumulable json - 
applications/local-1426533911241/stages/0/0,
+rdd list storage json - 
applications/local-1422981780767/storage/rdd,
+one rdd storage json - 
applications/local-1422981780767/storage/rdd/0
+// TODO multi-attempt stages
+  )
+
+  // run a bunch of characterization 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29090297
  
--- Diff: 
core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala ---
@@ -14,22 +14,170 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.spark.deploy.history
 
-import javax.servlet.http.HttpServletRequest
-
-import scala.collection.mutable
+import java.io.{File, FileInputStream, FileWriter, IOException}
+import java.net.{HttpURLConnection, URL}
+import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
 
-import org.apache.hadoop.fs.Path
-import org.mockito.Mockito.{when}
-import org.scalatest.FunSuite
-import org.scalatest.Matchers
+import org.apache.commons.io.{FileUtils, IOUtils}
+import org.mockito.Mockito.when
+import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
 import org.scalatest.mock.MockitoSugar
 
+import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf}
 import org.apache.spark.ui.SparkUI
 
-class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar {
+class HistoryServerSuite extends FunSuite with BeforeAndAfter with 
Matchers with MockitoSugar
+  with JsonTestUtils {
+
+  private val logDir = new File(src/test/resources/spark-events)
+  private val expRoot = new 
File(src/test/resources/HistoryServerExpectations/)
+
+  private var provider: FsHistoryProvider = null
+  private var server: HistoryServer = null
+  private var port: Int = -1
+
+  def init(): Unit = {
+val conf = new SparkConf()
+  .set(spark.history.fs.logDirectory, logDir.getAbsolutePath)
+  .set(spark.history.fs.updateInterval, 0)
+  .set(spark.testing, true)
+provider = new FsHistoryProvider(conf)
+provider.checkForLogs()
+val securityManager = new SecurityManager(conf)
+
+server = new HistoryServer(conf, provider, securityManager, 18080)
+server.initialize()
+server.bind()
+port = server.boundPort
+  }
+  def stop(): Unit = {
--- End diff --

nit: add empty line


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089846
  
--- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
@@ -0,0 +1,253 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the License); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an AS IS BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.Date
+
+import scala.collection.Map
+
+import org.apache.spark.JobExecutionStatus
+
+class ApplicationInfo(
--- End diff --

same question as with enums: do we want to make these public right away?

I'm not against it, just wondering if this was intentional.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089792
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala ---
@@ -0,0 +1,55 @@
+/*
+ * 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 java.text.SimpleDateFormat
+import java.util.TimeZone
+import javax.ws.rs.WebApplicationException
+import javax.ws.rs.core.Response
+import javax.ws.rs.core.Response.Status
+
+import scala.util.Try
+
+private[v1] class SimpleDateParam(val originalValue: String) {
+  val timestamp: Long = {
+SimpleDateParam.formats.collectFirst {
+  case fmt if Try{ fmt.parse(originalValue) }.isSuccess =
--- End diff --

nit: `Try(fmt.parse(originalValue))`


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29090268
  
--- Diff: 
core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala ---
@@ -14,22 +14,170 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.spark.deploy.history
 
-import javax.servlet.http.HttpServletRequest
-
-import scala.collection.mutable
+import java.io.{File, FileInputStream, FileWriter, IOException}
+import java.net.{HttpURLConnection, URL}
+import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
 
-import org.apache.hadoop.fs.Path
-import org.mockito.Mockito.{when}
-import org.scalatest.FunSuite
-import org.scalatest.Matchers
+import org.apache.commons.io.{FileUtils, IOUtils}
+import org.mockito.Mockito.when
+import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
 import org.scalatest.mock.MockitoSugar
 
+import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf}
 import org.apache.spark.ui.SparkUI
 
-class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar {
+class HistoryServerSuite extends FunSuite with BeforeAndAfter with 
Matchers with MockitoSugar
+  with JsonTestUtils {
+
+  private val logDir = new File(src/test/resources/spark-events)
+  private val expRoot = new 
File(src/test/resources/HistoryServerExpectations/)
+
+  private var provider: FsHistoryProvider = null
+  private var server: HistoryServer = null
+  private var port: Int = -1
+
+  def init(): Unit = {
+val conf = new SparkConf()
+  .set(spark.history.fs.logDirectory, logDir.getAbsolutePath)
+  .set(spark.history.fs.updateInterval, 0)
+  .set(spark.testing, true)
+provider = new FsHistoryProvider(conf)
+provider.checkForLogs()
+val securityManager = new SecurityManager(conf)
+
+server = new HistoryServer(conf, provider, securityManager, 18080)
+server.initialize()
+server.bind()
+port = server.boundPort
+  }
+  def stop(): Unit = {
+server.stop()
+  }
+
+  before {
+init()
+  }
+
+  after{
+stop()
+  }
+
+  val cases = Seq(
+application list json - applications,
+completed app list json - applications?status=completed,
+running app list json - applications?status=running,
+minDate app list json - applications?minDate=2015-02-10,
+maxDate app list json - applications?maxDate=2015-02-10,
+maxDate2 app list json - 
applications?maxDate=2015-02-03T10:42:40.000CST,
+one app json - applications/local-1422981780767,
+job list json - applications/local-1422981780767/jobs,
+one job json - applications/local-1422981780767/jobs/0,
+succeeded job list json - 
applications/local-1422981780767/jobs?status=succeeded,
+succeededfailed job list json -
+  
applications/local-1422981780767/jobs?status=succeededstatus=failed,
+executor list json - applications/local-1422981780767/executors,
+stage list json - applications/local-1422981780767/stages,
+complete stage list json - 
applications/local-1422981780767/stages?status=complete,
+failed stage list json - 
applications/local-1422981780767/stages?status=failed,
+one stage json - applications/local-1422981780767/stages/1,
+one stage attempt json - 
applications/local-1422981780767/stages/1/0,
+
+stage task summary - 
applications/local-1427397477963/stages/20/0/taskSummary,
+stage task summary w/ custom quantiles -
+  
applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99,
+
+stage task list - 
applications/local-1427397477963/stages/20/0/taskList,
+stage task list w/ offset  length -
+  
applications/local-1427397477963/stages/20/0/taskList?offset=10length=50,
+stage task list w/ sortBy -
+  
applications/local-1427397477963/stages/20/0/taskList?sortBy=DecreasingRuntime,
+stage task list w/ sortBy short names: -runtime -
+  
applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime,
+stage task list w/ sortBy short names: runtime -
+  
applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime,
+
+stage list with accumulable json - 
applications/local-1426533911241/stages,
+stage with accumulable json - 
applications/local-1426533911241/stages/0/0,
+rdd list storage json - 
applications/local-1422981780767/storage/rdd,
+one rdd storage json - 
applications/local-1422981780767/storage/rdd/0
+// TODO multi-attempt stages
+  )
+
+  // run a bunch of characterization 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089553
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
@@ -0,0 +1,283 @@
+/*
+ * 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 java.util.Date
+import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.executor.{InputMetrics = InternalInputMetrics, 
OutputMetrics = InternalOutputMetrics, ShuffleReadMetrics = 
InternalShuffleReadMetrics, ShuffleWriteMetrics = InternalShuffleWriteMetrics, 
TaskMetrics = InternalTaskMetrics}
+import org.apache.spark.scheduler.{AccumulableInfo = 
InternalAccumulableInfo, StageInfo}
+import org.apache.spark.status.api._
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+import org.apache.spark.util.Distribution
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllStagesResource(uiRoot: UIRoot) {
+
+  @GET
+  def stageList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[StageStatus]
--- End diff --

Same comment about using `JList`.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089570
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
@@ -0,0 +1,283 @@
+/*
+ * 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 java.util.Date
+import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.executor.{InputMetrics = InternalInputMetrics, 
OutputMetrics = InternalOutputMetrics, ShuffleReadMetrics = 
InternalShuffleReadMetrics, ShuffleWriteMetrics = InternalShuffleWriteMetrics, 
TaskMetrics = InternalTaskMetrics}
+import org.apache.spark.scheduler.{AccumulableInfo = 
InternalAccumulableInfo, StageInfo}
+import org.apache.spark.status.api._
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+import org.apache.spark.util.Distribution
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllStagesResource(uiRoot: UIRoot) {
+
+  @GET
+  def stageList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[StageStatus]
+  ): Seq[StageData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val listener = ui.jobProgressListener
+  val stageAndStatus = AllStagesResource.stagesAndStatus(ui)
+  val adjStatuses = {
+if (statuses.isEmpty()) {
+  java.util.Arrays.asList(StageStatus.values(): _*)
--- End diff --

Same about importing.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089653
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala
 ---
@@ -0,0 +1,81 @@
+/*
+ * 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 java.util.Date
+import java.util.{Arrays, List = JList}
--- End diff --

nit: you can merge the previous line with this one.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089315
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala ---
@@ -0,0 +1,107 @@
+/*
+ * 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 java.util
+import java.util.Date
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllJobsResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[JobExecutionStatus]
--- End diff --

instead of this, do:

import java.util.{Date, List = JList}

And use `JList` everywhere instead of `java.util.List` or `util.List` (both 
of which you're using in different places).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089352
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala ---
@@ -0,0 +1,107 @@
+/*
+ * 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 java.util
+import java.util.Date
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllJobsResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[JobExecutionStatus]
+  ): Seq[JobData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] =
+AllJobsResource.getStatusToJobs(ui)
+  val adjStatuses: util.List[JobExecutionStatus] = {
+if (statuses.isEmpty) {
+  java.util.Arrays.asList(JobExecutionStatus.values(): _*)
+}
+else {
--- End diff --

nit: `else` goes in the previous line


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29090015
  
--- Diff: core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala 
---
@@ -22,27 +22,28 @@ import javax.servlet.http.HttpServletRequest
 
 import scala.xml.Node
 
+import org.apache.spark.status.api.v1.ExecutorSummary
 import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage}
 import org.apache.spark.util.Utils
 
-/** Summary information about an executor to display in the UI. */
-// Needs to be private[ui] because of a false positive MiMa failure.
+// This isn't even used anymore -- but we need to keep it b/c of a MiMa 
false positive
 private[ui] case class ExecutorSummaryInfo(
-id: String,
-hostPort: String,
-rddBlocks: Int,
-memoryUsed: Long,
-diskUsed: Long,
-activeTasks: Int,
-failedTasks: Int,
-completedTasks: Int,
-totalTasks: Int,
-totalDuration: Long,
-totalInputBytes: Long,
-totalShuffleRead: Long,
-totalShuffleWrite: Long,
-maxMemory: Long,
-executorLogs: Map[String, String])
+  id: String,
--- End diff --

nit: the previous indentation level was correct.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-96081014
  
Alright, I think I finished my review. Looking good, just a bunch of nits 
and some dependency stuff to sort out. You could probably also remove the WIP 
from the title at this point.

This might conflict with #5432 but hopefully won't be too bad.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29087992
  
--- Diff: core/pom.xml ---
@@ -220,6 +220,21 @@
   version3.2.10/version
 /dependency
 dependency
+  groupIdcom.fasterxml.jackson.module/groupId
+  artifactIdjackson-module-scala_2.10/artifactId
+  version2.3.1/version
+/dependency
+dependency
+  groupIdcom.sun.jersey/groupId
+  artifactIdjersey-server/artifactId
+  version1.9/version
--- End diff --

Since we want this dependency to track the one used by YARN (to avoid 
adding more dependencies), can you add `scope${hadoop.deps.scope}/scope` 
here?

It's a not-so-explicit indication that this is the same version that hadoop 
uses (and helps people who don't package hadoop classes with Spark).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29088049
  
--- Diff: core/pom.xml ---
@@ -215,6 +215,11 @@
   version3.2.10/version
 /dependency
 dependency
+  groupIdcom.fasterxml.jackson.module/groupId
+  artifactIdjackson-module-scala_2.10/artifactId
+  version2.3.1/version
--- End diff --

@squito just a reminder that you should update this version to 
`${fasterxml.jackson.version}`.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29088868
  
--- Diff: 
core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java ---
@@ -0,0 +1,45 @@
+package org.apache.spark.status.api.v1;/*
+ * 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.
+ */
+
+import org.apache.spark.status.api.EnumUtil;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public enum TaskSorting {
--- End diff --

These enums don't seem to be used outside of this package; could we get 
away with making them package-private, or will jackson/jersey complain?

If that's not possible, we should probably tag them with `@DeveloperApi`, 
at least until we decide to make the non-REST API public also.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29088981
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
@@ -43,7 +43,7 @@ import org.apache.spark.{Logging, SecurityManager, 
SparkConf}
 private[history] class FsHistoryProvider(conf: SparkConf) extends 
ApplicationHistoryProvider
   with Logging {
 
-  import FsHistoryProvider._
+  import org.apache.spark.deploy.history.FsHistoryProvider._
--- End diff --

nit: this seems like your IDE ran through this code, but this (and the 
other import changes) are not really necessary.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089055
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala ---
@@ -93,7 +93,8 @@ private[history] class HistoryPage(parent: HistoryServer) 
extends WebUIPage()
 } else if (requestedIncomplete) {
   h4No incomplete applications found!/h4
 } else {
-  h4No completed applications found!/h4 ++
+  h4No {if (requestedIncomplete) running else completed}
--- End diff --

You can revert all changes to this file. This issue has been fixed 
separately (as you can see in the condition in L93).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089007
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
@@ -136,7 +136,9 @@ private[history] class FsHistoryProvider(conf: 
SparkConf) extends ApplicationHis
 }
   }
 
-  override def getListing(): Iterable[FsApplicationHistoryInfo] = 
applications.values
+  override def getListing(): Iterable[FsApplicationHistoryInfo] = {
--- End diff --

nit: change is not really necessary


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089246
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala ---
@@ -0,0 +1,107 @@
+/*
+ * 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 java.util
--- End diff --

not needed?


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

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




[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-24 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r29089370
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala ---
@@ -0,0 +1,107 @@
+/*
+ * 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 java.util
+import java.util.Date
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllJobsResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[JobExecutionStatus]
+  ): Seq[JobData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] =
+AllJobsResource.getStatusToJobs(ui)
+  val adjStatuses: util.List[JobExecutionStatus] = {
+if (statuses.isEmpty) {
+  java.util.Arrays.asList(JobExecutionStatus.values(): _*)
--- End diff --

nit: import `Arrays` at the top


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-23 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-95749967
  
  [Test build #30883 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30883/consoleFull)
 for   PR 4435 at commit 
[`f2e63ad`](https://github.com/apache/spark/commit/f2e63ad7ca2fc8f9490f80a8c059f7ae8d4046a9).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-23 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-95771453
  
  [Test build #30883 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30883/consoleFull)
 for   PR 4435 at commit 
[`f2e63ad`](https://github.com/apache/spark/commit/f2e63ad7ca2fc8f9490f80a8c059f7ae8d4046a9).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.
 * This patch **adds the following new dependencies:**
   * `aopalliance-repackaged-2.4.0-b06.jar`
   * `asm-3.1.jar`
   * `hk2-api-2.4.0-b06.jar`
   * `hk2-locator-2.4.0-b06.jar`
   * `hk2-utils-2.4.0-b06.jar`
   * `jackson-annotations-2.4.4.jar`
   * `jackson-jaxrs-base-2.3.2.jar`
   * `jackson-jaxrs-json-provider-2.3.2.jar`
   * `jackson-module-jaxb-annotations-2.3.2.jar`
   * `javassist-3.18.1-GA.jar`
   * `javax.annotation-api-1.2.jar`
   * `javax.inject-2.4.0-b06.jar`
   * `javax.ws.rs-api-2.0.1.jar`
   * `jersey-common-2.15.jar`
   * `jersey-guava-2.15.jar`
   * `jersey-media-json-jackson-2.15.jar`
   * `osgi-resource-locator-1.0.1.jar`

 * This patch **removes the following dependencies:**
   * `jackson-annotations-2.4.0.jar`



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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-23 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-95771474
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30883/
Test PASSed.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731362
  
--- Diff: core/src/main/java/org/apache/spark/status/api/EnumUtil.java ---
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import com.google.common.base.Joiner;
+
+import java.util.Arrays;
+
+public class EnumUtil {
+  public static E extends EnumE E parseIgnoreCase(ClassE clz, String 
str) {
+E[] constants = clz.getEnumConstants();
+if (str == null) {
+  return null;
+}
+for (E e : constants) {
+  if (e.name().equalsIgnoreCase(str))
+return e;
+}
+throw new IllegalArgumentException(
+  String.format(Illegal type='%s'. Supported type values: %s,
+str, Joiner.on(, ).join(
+  Arrays.asList(constants;
--- End diff --

Is `Arrays.asList` needed? There's a `join(Object[])` method in Joiner.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731820
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
@@ -112,6 +118,10 @@ class HistoryServer(
*/
   def initialize() {
 attachPage(new HistoryPage(this))
+
--- End diff --

nit: delete


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28732740
  
--- Diff: core/pom.xml ---
@@ -220,6 +220,21 @@
   version3.2.10/version
 /dependency
 dependency
+  groupIdcom.fasterxml.jackson.module/groupId
+  artifactIdjackson-module-scala_2.10/artifactId
+  version2.3.1/version
+/dependency
+dependency
+  groupIdcom.sun.jersey/groupId
+  artifactIdjersey-server/artifactId
+  version1.9/version
+/dependency
+dependency
+  groupIdorg.glassfish.jersey.media/groupId
+  artifactIdjersey-media-json-jackson/artifactId
--- End diff --

In fact is this needed at all? It seems to be a plugin for jersey 2.x which 
is not what you're using.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731264
  
--- Diff: core/src/main/java/org/apache/spark/status/api/EnumUtil.java ---
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import com.google.common.base.Joiner;
+
+import java.util.Arrays;
+
+public class EnumUtil {
+  public static E extends EnumE E parseIgnoreCase(ClassE clz, String 
str) {
+E[] constants = clz.getEnumConstants();
+if (str == null) {
+  return null;
+}
+for (E e : constants) {
+  if (e.name().equalsIgnoreCase(str))
--- End diff --

nit:

if {
  ...
}


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731238
  
--- Diff: core/src/main/java/org/apache/spark/status/api/EnumUtil.java ---
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import com.google.common.base.Joiner;
+
+import java.util.Arrays;
+
+public class EnumUtil {
--- End diff --

Also... if this is supposed to be a helper class, maybe it should be in 
different package. If it's supposed to be used by this code only, maybe it 
could be package-private?


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28732087
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala ---
@@ -149,7 +159,13 @@ class HistoryServer(
*
* @return List of all known applications.
*/
-  def getApplicationList(): Iterable[ApplicationHistoryInfo] = 
provider.getListing()
+  def getApplicationList(): Iterable[ApplicationHistoryInfo] = {
+provider.getListing()
+  }
+
+  def getApplicationInfoList: Seq[ApplicationInfo] = {
+getApplicationList().map { 
ApplicationsListResource.appHistoryInfoToPublicAppInfo }.toSeq
--- End diff --

Is this transformation lazy? Otherwise it can cause memory usage to shoot 
up considerably when this method is called (which can happen from multiple 
concurrent user requests).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731684
  
--- Diff: 
core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java ---
@@ -0,0 +1,45 @@
+package org.apache.spark.status.api.v1;/*
+ * 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.
+ */
+
+import org.apache.spark.status.api.EnumUtil;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public enum TaskSorting {
+  ID,
+  IncreasingRuntime(runtime),
+  DecreasingRuntime(-runtime);
+
+  final SetString alternateNames;
+  TaskSorting(String... names) {
+alternateNames = new HashSetString();
+for (String n: names) {
+  alternateNames.add(n);
+}
+  }
+
+  public static TaskSorting fromString(String str) {
+for (TaskSorting t: values()) {
+  if (t.alternateNames.contains(str.toLowerCase())) {
--- End diff --

nit: you could hoist `str.toLowerCase()` out of the loop.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731144
  
--- Diff: core/src/main/java/org/apache/spark/status/api/EnumUtil.java ---
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import com.google.common.base.Joiner;
+
+import java.util.Arrays;
+
+public class EnumUtil {
--- End diff --

Tag this as `@DeveloperApi`? Kinda sucks that Java doesn't have the 
`private[foo]` modifier.

(I'm trying to avoid suggesting using Hadoop's `InterfaceAudience` 
annotation...)


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731520
  
--- Diff: 
core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java ---
@@ -0,0 +1,31 @@
+/*
+ * 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 org.apache.spark.status.api.EnumUtil;
+
+public enum StageStatus {
+  Active,
--- End diff --

Any reason for these to not be `ALL_CAPS` like, e.g., `ApplicationStatus`?


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731535
  
--- Diff: 
core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java ---
@@ -0,0 +1,45 @@
+package org.apache.spark.status.api.v1;/*
--- End diff --

Oops.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731581
  
--- Diff: 
core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java ---
@@ -0,0 +1,45 @@
+package org.apache.spark.status.api.v1;/*
+ * 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.
+ */
+
+import org.apache.spark.status.api.EnumUtil;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public enum TaskSorting {
+  ID,
+  IncreasingRuntime(runtime),
+  DecreasingRuntime(-runtime);
+
+  final SetString alternateNames;
--- End diff --

private


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28731597
  
--- Diff: 
core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java ---
@@ -0,0 +1,45 @@
+package org.apache.spark.status.api.v1;/*
+ * 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.
+ */
+
+import org.apache.spark.status.api.EnumUtil;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public enum TaskSorting {
+  ID,
+  IncreasingRuntime(runtime),
+  DecreasingRuntime(-runtime);
+
+  final SetString alternateNames;
+  TaskSorting(String... names) {
--- End diff --

private


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28732381
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala ---
@@ -0,0 +1,111 @@
+/*
+ * 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.ws.rs.{GET, PathParam, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageUtils}
+import org.apache.spark.ui.storage.StorageListener
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllRDDResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String
+  ): Seq[RDDStorageInfo] = {
+uiRoot.withSparkUI(appId) { ui =
+  // should all access on storageListener also be synchronized?
+  val storageStatusList = ui.storageListener.storageStatusList
+  val rddInfos = ui.storageListener.rddInfoList
+  rddInfos.map{rddInfo =
+AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, 
storageStatusList,
+  includeDetails = false)
+  }
+
+}
+  }
+
+}
+
+private[spark] object AllRDDResource {
+
+  def getRDDStorageInfo(
+  rddId: Int,
+  listener: StorageListener,
+  includeDetails: Boolean): Option[RDDStorageInfo] = {
+val storageStatusList = listener.storageStatusList
+listener.rddInfoList.find { _.id == rddId }.map { rddInfo =
+  getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails)
+}
+  }
+
+  def getRDDStorageInfo(
+rddId: Int,
+rddInfo: RDDInfo,
+storageStatusList: Seq[StorageStatus],
+includeDetails: Boolean
+  ): RDDStorageInfo = {
+val workers = storageStatusList.map { (rddId, _) }
+val blockLocations = StorageUtils.getRddBlockLocations(rddId, 
storageStatusList)
+val blocks = storageStatusList
+  .flatMap { _.rddBlocksById(rddId) }
+  .sortWith { _._1.name  _._1.name }
+  .map { case (blockId, status) =
+(blockId, status, 
blockLocations.get(blockId).getOrElse(Seq[String](Unknown)))
+  }
+
--- End diff --

nit: delete


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-20 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r28730899
  
--- Diff: core/pom.xml ---
@@ -220,6 +220,21 @@
   version3.2.10/version
 /dependency
 dependency
+  groupIdcom.fasterxml.jackson.module/groupId
+  artifactIdjackson-module-scala_2.10/artifactId
+  version2.3.1/version
+/dependency
+dependency
+  groupIdcom.sun.jersey/groupId
+  artifactIdjersey-server/artifactId
+  version1.9/version
+/dependency
+dependency
+  groupIdorg.glassfish.jersey.media/groupId
+  artifactIdjersey-media-json-jackson/artifactId
--- End diff --

Could you double-check that pulling this is not causing different 
com.fasterxml.jackson libraries to be mixed up? I checked the pom for this 
artifact and it seems to depend on a different version of jackson than the one 
Spark pulls.

Also, I wonder if it wouldn't be better to do like Yarn and avoid pulling 
this dependency (which is from a different Jersey version). See:

https://github.com/apache/hadoop/blob/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnJacksonJaxbJsonProvider.java


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-17 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93898160
  
I'd like to finish reviewing this, but I keep getting pre-empted by other 
work, so instead I'll leave a list of things that I would look at / check when 
reviewing this (to let other folks pick up and finish the review).  This looks 
like it's in pretty good shape overall, though, so hopefully it won't be too 
much work to finish this.

Here's what I'd look at in any final review passes:

- Has the visibility of new classes / methods / interfaces been restricted 
to the narrowest possible scope (i.e. are we unintentionally exposing internal 
functionality)?  If something _has_ to be public but is not intended to be 
stable / available to users, we should add a documentation comment to explain 
this.
- Have accesses to listeners been properly synchronized?
- Are there any code style nits that we should clean up?  I noticed a bunch 
of minor indentation problems, but don't really have time to comment 
individually.
- I'd take a look at how we handle timestamps in JSON, just to double-check 
that we're exposing them in an easy-to-consume format.
- Documentation-wise, are there any confusing parts of the code that need 
to be documented?
- Can we add a top-level Javadoc comment somewhere to explain our overall 
strategy for handling JSON compatibility, etc, and maybe a checklist / rules to 
follow when changing these classes?  There's something similar to this in one 
of the JSONProtocol classes, which might be nice to model this on.

I'd also manually test this in a spark-shell.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-17 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93889612
  
Hey @squito it looks like the automated dependency checking isn't working 
so well for this PR. Can you do a diff and list all of the dependencies this is 
adding to or updating in Spark? Creating conflicts with user applications seems 
like a concern with this patch. Right now the patch shades the asm dependency, 
is there any reason to shade that one in particular and not others?


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-15 Thread squito
Github user squito commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93369479
  
jenkins, retest this please


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-15 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93372085
  
  [Test build #30339 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30339/consoleFull)
 for   PR 4435 at commit 
[`c22b11f`](https://github.com/apache/spark/commit/c22b11f0a808135e492cb50c5b5bdebcfd73b1a5).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-15 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93434588
  
  [Test build #30339 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30339/consoleFull)
 for   PR 4435 at commit 
[`c22b11f`](https://github.com/apache/spark/commit/c22b11f0a808135e492cb50c5b5bdebcfd73b1a5).
 * This patch **passes all tests**.
 * This patch **does not merge cleanly**.
 * This patch adds the following public classes _(experimental)_:
  * `public class EnumUtil `
  * `abstract class MetricHelper[I,O](f: InternalTaskMetrics = 
Option[I]) `
  * `class ApplicationInfo(`
  * `class ExecutorStageSummary(`
  * `class ExecutorSummary(`
  * `class JobData(`
  * `class RDDStorageInfo(`
  * `class RDDDataDistribution(`
  * `class RDDPartitionInfo(`
  * `class StageData(`
  * `class TaskData(`
  * `class TaskMetrics(`
  * `class InputMetrics(`
  * `class OutputMetrics(`
  * `class ShuffleReadMetrics(`
  * `class ShuffleWriteMetrics(`
  * `class TaskMetricDistributions(`
  * `class InputMetricDistributions(`
  * `class OutputMetricDistributions(`
  * `class ShuffleReadMetricDistributions(`
  * `class ShuffleWriteMetricDistributions(`
  * `class AccumulableInfo (`

 * This patch **adds the following new dependencies:**
   * `aopalliance-repackaged-2.4.0-b06.jar`
   * `asm-3.1.jar`
   * `commons-math3-3.1.1.jar`
   * `hk2-api-2.4.0-b06.jar`
   * `hk2-locator-2.4.0-b06.jar`
   * `hk2-utils-2.4.0-b06.jar`
   * `jackson-annotations-2.4.4.jar`
   * `jackson-jaxrs-base-2.3.2.jar`
   * `jackson-jaxrs-json-provider-2.3.2.jar`
   * `jackson-module-jaxb-annotations-2.3.2.jar`
   * `javassist-3.18.1-GA.jar`
   * `javax.annotation-api-1.2.jar`
   * `javax.inject-2.4.0-b06.jar`
   * `javax.ws.rs-api-2.0.1.jar`
   * `jersey-common-2.15.jar`
   * `jersey-guava-2.15.jar`
   * `jersey-media-json-jackson-2.15.jar`
   * `osgi-resource-locator-1.0.1.jar`
   * `snappy-java-1.1.1.6.jar`

 * This patch **removes the following dependencies:**
   * `commons-math3-3.4.1.jar`
   * `jackson-annotations-2.4.0.jar`
   * `snappy-java-1.1.1.7.jar`



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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93434620
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30339/
Test PASSed.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-15 Thread squito
Github user squito commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93446557
  
updated to go back to java enums


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-15 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93217736
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30301/
Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-15 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93217730
  
**[Test build #30301 timed 
out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30301/consoleFull)**
 for PR 4435 at commit 
[`c22b11f`](https://github.com/apache/spark/commit/c22b11f0a808135e492cb50c5b5bdebcfd73b1a5)
 after a configured wait of `120m`.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-14 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93156365
  
  [Test build #30294 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30294/consoleFull)
 for   PR 4435 at commit 
[`9ea682c`](https://github.com/apache/spark/commit/9ea682c91b4a05b5d9eb19aa163299d6ea67838b).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-14 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93179548
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30294/
Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-14 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93179544
  
  [Test build #30294 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30294/consoleFull)
 for   PR 4435 at commit 
[`9ea682c`](https://github.com/apache/spark/commit/9ea682c91b4a05b5d9eb19aa163299d6ea67838b).
 * This patch **fails Spark unit tests**.
 * This patch **does not merge cleanly**.
 * This patch adds the following public classes _(experimental)_:
  * `public class EnumUtil `
  * `abstract class MetricHelper[I,O](f: InternalTaskMetrics = 
Option[I]) `
  * `class ApplicationInfo(`
  * `class ExecutorStageSummary(`
  * `class ExecutorSummary(`
  * `class JobData(`
  * `class RDDStorageInfo(`
  * `class RDDDataDistribution(`
  * `class RDDPartitionInfo(`
  * `class StageData(`
  * `class TaskData(`
  * `class TaskMetrics(`
  * `class InputMetrics(`
  * `class OutputMetrics(`
  * `class ShuffleReadMetrics(`
  * `class ShuffleWriteMetrics(`
  * `class TaskMetricDistributions(`
  * `class InputMetricDistributions(`
  * `class OutputMetricDistributions(`
  * `class ShuffleReadMetricDistributions(`
  * `class ShuffleWriteMetricDistributions(`
  * `class AccumulableInfo (`

 * This patch **removes the following dependencies:**
   * `RoaringBitmap-0.4.5.jar`
   * `activation-1.1.jar`
   * `akka-actor_2.10-2.3.4-spark.jar`
   * `akka-remote_2.10-2.3.4-spark.jar`
   * `akka-slf4j_2.10-2.3.4-spark.jar`
   * `aopalliance-1.0.jar`
   * `arpack_combined_all-0.1.jar`
   * `avro-1.7.7.jar`
   * `breeze-macros_2.10-0.11.2.jar`
   * `breeze_2.10-0.11.2.jar`
   * `chill-java-0.5.0.jar`
   * `chill_2.10-0.5.0.jar`
   * `commons-beanutils-1.7.0.jar`
   * `commons-beanutils-core-1.8.0.jar`
   * `commons-cli-1.2.jar`
   * `commons-codec-1.10.jar`
   * `commons-collections-3.2.1.jar`
   * `commons-compress-1.4.1.jar`
   * `commons-configuration-1.6.jar`
   * `commons-digester-1.8.jar`
   * `commons-httpclient-3.1.jar`
   * `commons-io-2.1.jar`
   * `commons-lang-2.5.jar`
   * `commons-lang3-3.3.2.jar`
   * `commons-math-2.1.jar`
   * `commons-math3-3.4.1.jar`
   * `commons-net-2.2.jar`
   * `compress-lzf-1.0.0.jar`
   * `config-1.2.1.jar`
   * `core-1.1.2.jar`
   * `curator-client-2.4.0.jar`
   * `curator-framework-2.4.0.jar`
   * `curator-recipes-2.4.0.jar`
   * `gmbal-api-only-3.0.0-b023.jar`
   * `grizzly-framework-2.1.2.jar`
   * `grizzly-http-2.1.2.jar`
   * `grizzly-http-server-2.1.2.jar`
   * `grizzly-http-servlet-2.1.2.jar`
   * `grizzly-rcm-2.1.2.jar`
   * `groovy-all-2.3.7.jar`
   * `guava-14.0.1.jar`
   * `guice-3.0.jar`
   * `hadoop-annotations-2.2.0.jar`
   * `hadoop-auth-2.2.0.jar`
   * `hadoop-client-2.2.0.jar`
   * `hadoop-common-2.2.0.jar`
   * `hadoop-hdfs-2.2.0.jar`
   * `hadoop-mapreduce-client-app-2.2.0.jar`
   * `hadoop-mapreduce-client-common-2.2.0.jar`
   * `hadoop-mapreduce-client-core-2.2.0.jar`
   * `hadoop-mapreduce-client-jobclient-2.2.0.jar`
   * `hadoop-mapreduce-client-shuffle-2.2.0.jar`
   * `hadoop-yarn-api-2.2.0.jar`
   * `hadoop-yarn-client-2.2.0.jar`
   * `hadoop-yarn-common-2.2.0.jar`
   * `hadoop-yarn-server-common-2.2.0.jar`
   * `ivy-2.4.0.jar`
   * `jackson-annotations-2.4.0.jar`
   * `jackson-core-2.4.4.jar`
   * `jackson-core-asl-1.8.8.jar`
   * `jackson-databind-2.4.4.jar`
   * `jackson-jaxrs-1.8.8.jar`
   * `jackson-mapper-asl-1.8.8.jar`
   * `jackson-module-scala_2.10-2.4.4.jar`
   * `jackson-xc-1.8.8.jar`
   * `jansi-1.4.jar`
   * `javax.inject-1.jar`
   * `javax.servlet-3.0.0.v201112011016.jar`
   * `javax.servlet-3.1.jar`
   * `javax.servlet-api-3.0.1.jar`
   * `jaxb-api-2.2.2.jar`
   * `jaxb-impl-2.2.3-1.jar`
   * `jcl-over-slf4j-1.7.10.jar`
   * `jersey-client-1.9.jar`
   * `jersey-core-1.9.jar`
   * `jersey-grizzly2-1.9.jar`
   * `jersey-guice-1.9.jar`
   * `jersey-json-1.9.jar`
   * `jersey-server-1.9.jar`
   * `jersey-test-framework-core-1.9.jar`
   * `jersey-test-framework-grizzly2-1.9.jar`
   * `jets3t-0.7.1.jar`
   * `jettison-1.1.jar`
   * `jetty-util-6.1.26.jar`
   * `jline-0.9.94.jar`
   * `jline-2.10.4.jar`
   * `jodd-core-3.6.3.jar`
   * `json4s-ast_2.10-3.2.10.jar`
   * `json4s-core_2.10-3.2.10.jar`
   * `json4s-jackson_2.10-3.2.10.jar`
   * `jsr305-1.3.9.jar`
   * `jtransforms-2.4.0.jar`
   * `jul-to-slf4j-1.7.10.jar`
   * `kryo-2.21.jar`
   * `log4j-1.2.17.jar`
   * `lz4-1.2.0.jar`
   * `management-api-3.0.0-b012.jar`
   * `mesos-0.21.0-shaded-protobuf.jar`
   * `metrics-core-3.1.0.jar`
   * 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-14 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-93180574
  
  [Test build #30301 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30301/consoleFull)
 for   PR 4435 at commit 
[`c22b11f`](https://github.com/apache/spark/commit/c22b11f0a808135e492cb50c5b5bdebcfd73b1a5).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-92460625
  
  [Test build #30182 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30182/consoleFull)
 for   PR 4435 at commit 
[`d493b38`](https://github.com/apache/spark/commit/d493b384de505ce5cec0a4cbbd21490b9abba419).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-92462673
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30182/
Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-92514829
  
  [Test build #30191 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30191/consoleFull)
 for   PR 4435 at commit 
[`cf86175`](https://github.com/apache/spark/commit/cf86175c3d9f8d4263a3ac8461e50c797590f349).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `abstract class MetricHelper[I,O](f: InternalTaskMetrics = 
Option[I]) `
  * `sealed abstract class TaskSorting extends SparkEnum `
  * `class ApplicationInfo(`
  * `class ExecutorStageSummary(`
  * `class ExecutorSummary(`
  * `class JobData(`
  * `class RDDStorageInfo(`
  * `class RDDDataDistribution(`
  * `class RDDPartitionInfo(`
  * `class StageData(`
  * `class TaskData(`
  * `class TaskMetrics(`
  * `class InputMetrics(`
  * `class OutputMetrics(`
  * `class ShuffleReadMetrics(`
  * `class ShuffleWriteMetrics(`
  * `class TaskMetricDistributions(`
  * `class InputMetricDistributions(`
  * `class OutputMetricDistributions(`
  * `class ShuffleReadMetricDistributions(`
  * `class ShuffleWriteMetricDistributions(`
  * `class AccumulableInfo (`
  * `throw new SparkException(It appears you are using SparkEnum 
in a class which does not  +`
  * `  case class Assignment(id: Long, cluster: Int)`

 * This patch **removes the following dependencies:**
   * `RoaringBitmap-0.4.5.jar`
   * `activation-1.1.jar`
   * `akka-actor_2.10-2.3.4-spark.jar`
   * `akka-remote_2.10-2.3.4-spark.jar`
   * `akka-slf4j_2.10-2.3.4-spark.jar`
   * `aopalliance-1.0.jar`
   * `arpack_combined_all-0.1.jar`
   * `avro-1.7.7.jar`
   * `breeze-macros_2.10-0.11.2.jar`
   * `breeze_2.10-0.11.2.jar`
   * `chill-java-0.5.0.jar`
   * `chill_2.10-0.5.0.jar`
   * `commons-beanutils-1.7.0.jar`
   * `commons-beanutils-core-1.8.0.jar`
   * `commons-cli-1.2.jar`
   * `commons-codec-1.10.jar`
   * `commons-collections-3.2.1.jar`
   * `commons-compress-1.4.1.jar`
   * `commons-configuration-1.6.jar`
   * `commons-digester-1.8.jar`
   * `commons-httpclient-3.1.jar`
   * `commons-io-2.1.jar`
   * `commons-lang-2.5.jar`
   * `commons-lang3-3.3.2.jar`
   * `commons-math-2.1.jar`
   * `commons-math3-3.1.1.jar`
   * `commons-net-2.2.jar`
   * `compress-lzf-1.0.0.jar`
   * `config-1.2.1.jar`
   * `core-1.1.2.jar`
   * `curator-client-2.4.0.jar`
   * `curator-framework-2.4.0.jar`
   * `curator-recipes-2.4.0.jar`
   * `gmbal-api-only-3.0.0-b023.jar`
   * `grizzly-framework-2.1.2.jar`
   * `grizzly-http-2.1.2.jar`
   * `grizzly-http-server-2.1.2.jar`
   * `grizzly-http-servlet-2.1.2.jar`
   * `grizzly-rcm-2.1.2.jar`
   * `groovy-all-2.3.7.jar`
   * `guava-14.0.1.jar`
   * `guice-3.0.jar`
   * `hadoop-annotations-2.2.0.jar`
   * `hadoop-auth-2.2.0.jar`
   * `hadoop-client-2.2.0.jar`
   * `hadoop-common-2.2.0.jar`
   * `hadoop-hdfs-2.2.0.jar`
   * `hadoop-mapreduce-client-app-2.2.0.jar`
   * `hadoop-mapreduce-client-common-2.2.0.jar`
   * `hadoop-mapreduce-client-core-2.2.0.jar`
   * `hadoop-mapreduce-client-jobclient-2.2.0.jar`
   * `hadoop-mapreduce-client-shuffle-2.2.0.jar`
   * `hadoop-yarn-api-2.2.0.jar`
   * `hadoop-yarn-client-2.2.0.jar`
   * `hadoop-yarn-common-2.2.0.jar`
   * `hadoop-yarn-server-common-2.2.0.jar`
   * `ivy-2.4.0.jar`
   * `jackson-annotations-2.4.0.jar`
   * `jackson-core-2.4.4.jar`
   * `jackson-core-asl-1.8.8.jar`
   * `jackson-databind-2.4.4.jar`
   * `jackson-jaxrs-1.8.8.jar`
   * `jackson-mapper-asl-1.8.8.jar`
   * `jackson-module-scala_2.10-2.4.4.jar`
   * `jackson-xc-1.8.8.jar`
   * `jansi-1.4.jar`
   * `javax.inject-1.jar`
   * `javax.servlet-3.0.0.v201112011016.jar`
   * `javax.servlet-3.1.jar`
   * `javax.servlet-api-3.0.1.jar`
   * `jaxb-api-2.2.2.jar`
   * `jaxb-impl-2.2.3-1.jar`
   * `jcl-over-slf4j-1.7.10.jar`
   * `jersey-client-1.9.jar`
   * `jersey-core-1.9.jar`
   * `jersey-grizzly2-1.9.jar`
   * `jersey-guice-1.9.jar`
   * `jersey-json-1.9.jar`
   * `jersey-server-1.9.jar`
   * `jersey-test-framework-core-1.9.jar`
   * `jersey-test-framework-grizzly2-1.9.jar`
   * `jets3t-0.7.1.jar`
   * `jettison-1.1.jar`
   * `jetty-util-6.1.26.jar`
   * `jline-0.9.94.jar`
   * `jline-2.10.4.jar`
   * `jodd-core-3.6.3.jar`
   * `json4s-ast_2.10-3.2.10.jar`
   * `json4s-core_2.10-3.2.10.jar`
   * `json4s-jackson_2.10-3.2.10.jar`
   * `jsr305-1.3.9.jar`
   * `jtransforms-2.4.0.jar`
   * `jul-to-slf4j-1.7.10.jar`
   * `kryo-2.21.jar`
   * 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-92514838
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/30191/
Test PASSed.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-92490236
  
  [Test build #30191 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30191/consoleFull)
 for   PR 4435 at commit 
[`cf86175`](https://github.com/apache/spark/commit/cf86175c3d9f8d4263a3ac8461e50c797590f349).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-13 Thread squito
Github user squito commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-92550401
  
ping @JoshRosen .  I think this is basically ready to go.  The only open 
issue is figuring how to test the authentication for the new endpoints.  (I'm 
not even sure how to test it manually.)  I don't see any tests for 
authentication for the regular UI I can copy, however.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-92462664
  
  [Test build #30182 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/30182/consoleFull)
 for   PR 4435 at commit 
[`d493b38`](https://github.com/apache/spark/commit/d493b384de505ce5cec0a4cbbd21490b9abba419).
 * This patch **fails Scala style tests**.
 * This patch **does not merge cleanly**.
 * This patch adds the following public classes _(experimental)_:
  * `abstract class MetricHelper[I,O](f: InternalTaskMetrics = 
Option[I]) `
  * `sealed abstract class TaskSorting extends SparkEnum `
  * `class ApplicationInfo(`
  * `class ExecutorStageSummary(`
  * `class ExecutorSummary(`
  * `class JobData(`
  * `class RDDStorageInfo(`
  * `class RDDDataDistribution(`
  * `class RDDPartitionInfo(`
  * `class StageData(`
  * `class TaskData(`
  * `class TaskMetrics(`
  * `class InputMetrics(`
  * `class OutputMetrics(`
  * `class ShuffleReadMetrics(`
  * `class ShuffleWriteMetrics(`
  * `class TaskMetricDistributions(`
  * `class InputMetricDistributions(`
  * `class OutputMetricDistributions(`
  * `class ShuffleReadMetricDistributions(`
  * `class ShuffleWriteMetricDistributions(`
  * `class AccumulableInfo (`
  * `throw new SparkException(It appears you are using SparkEnum 
in a class which does not  +`
  * `  case class Assignment(id: Long, cluster: Int)`

 * This patch does not change any dependencies.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27981125
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala ---
@@ -0,0 +1,160 @@
+/*
+ * 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
+
+@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}/jobs)
+  def getJobs(): AllJobsResource = {
+new AllJobsResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/jobs/{jobId: \\d+})
+  def getJob(): OneJobResource = {
+new OneJobResource(uiRoot)
+  }
+
+
+  @Path(applications/{appId}/executors)
+  def getExecutors(): ExecutorListResource = {
+new ExecutorListResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/stages)
+  def getStages(): AllStagesResource= {
+new AllStagesResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/stages/{stageId: \\d+})
+  def getStage(): OneStageResource= {
+new OneStageResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/storage/rdd)
+  def getRdds(): AllRDDResource = {
+new AllRDDResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/storage/rdd/{rddId: \\d+})
+  def getRdd(): OneRDDResource = {
+new OneRDDResource(uiRoot)
+  }
+
+}
+
+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: Seq[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)(f: SparkUI = T): T = {
+getSparkUI(appId) 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 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27984083
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala ---
@@ -0,0 +1,192 @@
+/*
+ * 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.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.SparkException
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.StageUIData
+import org.apache.spark.util.SparkEnum
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneStageResource(uiRoot: UIRoot) {
+
+  @GET
+  @Path()
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int
+  ): Seq[StageData] = {
+forStage(appId, stageId){ (listener,stageAttempts) =
+  stageAttempts.map { case (status, stageInfo) =
+val stageUiData = listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId)).
+getOrElse(throw new SparkException(failed to get full stage 
data for stage:  +
--- End diff --

originally I thought 500 was OK, since it meant some internal inconsistent 
state.  But now I realize there is a little race condition between when I get 
the stage status, and the full stage ui data -- the lock on the listener is 
released in the middle, so the stage might get cleaned up in between.  I'll 
change the way `forStageAttempt` etc. to fix that


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27977857
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
@@ -0,0 +1,285 @@
+/*
+ * 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 java.util.Date
+import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.executor.{InputMetrics = InternalInputMetrics, 
OutputMetrics = InternalOutputMetrics, ShuffleReadMetrics = 
InternalShuffleReadMetrics, ShuffleWriteMetrics = InternalShuffleWriteMetrics, 
TaskMetrics = InternalTaskMetrics}
+import org.apache.spark.scheduler.{AccumulableInfo = 
InternalAccumulableInfo, StageInfo}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+import org.apache.spark.util.Distribution
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllStagesResource(uiRoot: UIRoot) {
+
+  @GET
+  def stageList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[StageStatus]
+  ): Seq[StageData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val listener = ui.jobProgressListener
+  val stageAndStatus = AllStagesResource.stagesAndStatus(ui)
+  val adjStatuses = {
+if (statuses.isEmpty()) {
+  java.util.Arrays.asList(StageStatus.values: _*)
+} else {
+  statuses
+}
+  }
+  for {
+(status, stageList) - stageAndStatus
+stageInfo: StageInfo - stageList if adjStatuses.contains(status)
+stageUiData: StageUIData - listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId))
+}
+  } yield {
+AllStagesResource.stageUiToStageData(status, stageInfo, 
stageUiData, includeDetails = false)
+  }
+}
+  }
+}
+
+private[v1] object AllStagesResource {
+  def stageUiToStageData(
+status: StageStatus,
+stageInfo: StageInfo,
+stageUiData: StageUIData,
+includeDetails: Boolean
+  ): StageData = {
+
+val taskData = if(includeDetails) {
+  Some(stageUiData.taskData.map { case (k, v) = k - 
convertTaskData(v) } )
+} else {
+  None
+}
+val executorSummary = if(includeDetails) {
+  Some(stageUiData.executorSummary.map { case (k, summary) =
+k - new ExecutorStageSummary(
+  taskTime = summary.taskTime,
+  failedTasks = summary.failedTasks,
+  succeededTasks = summary.succeededTasks,
+  inputBytes = summary.inputBytes,
+  outputBytes = summary.outputBytes,
+  shuffleRead = summary.shuffleRead,
+  shuffleWrite = summary.shuffleWrite,
+  memoryBytesSpilled = summary.memoryBytesSpilled,
+  diskBytesSpilled = summary.diskBytesSpilled
+)
+  })
+} else {
+  None
+}
+
+val accumulableInfo = stageUiData.accumulables.values.map { 
convertAccumulableInfo }.toSeq
+
+new StageData(
+  status = status,
+  stageId = stageInfo.stageId,
+  attemptId = stageInfo.attemptId,
+  numActiveTasks = stageUiData.numActiveTasks,
+  numCompleteTasks = stageUiData.numCompleteTasks,
+  numFailedTasks = stageUiData.numFailedTasks,
+  executorRunTime = stageUiData.executorRunTime,
+  inputBytes = stageUiData.inputBytes,
+  inputRecords = stageUiData.inputRecords,
+  outputBytes = stageUiData.outputBytes,
+  outputRecords = stageUiData.outputRecords,
+  shuffleReadBytes = stageUiData.shuffleReadTotalBytes,
+  shuffleReadRecords = stageUiData.shuffleReadRecords,
+  shuffleWriteBytes = stageUiData.shuffleWriteBytes,
+  shuffleWriteRecords = 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27992507
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala ---
@@ -0,0 +1,192 @@
+/*
+ * 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.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.SparkException
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.StageUIData
+import org.apache.spark.util.SparkEnum
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneStageResource(uiRoot: UIRoot) {
+
+  @GET
+  @Path()
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int
+  ): Seq[StageData] = {
+forStage(appId, stageId){ (listener,stageAttempts) =
+  stageAttempts.map { case (status, stageInfo) =
+val stageUiData = listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId)).
+getOrElse(throw new SparkException(failed to get full stage 
data for stage:  +
+stageInfo.stageId + : + stageInfo.attemptId)
+)
+}
+AllStagesResource.stageUiToStageData(status, stageInfo, 
stageUiData,
+  includeDetails = true)
+  }
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+})
+  def oneAttemptData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int
+  ): StageData = {
+forStageAttempt(appId, stageId, attemptId) { case (status, stageInfo, 
stageUiData) =
+  AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData,
+includeDetails = true)
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+}/taskSummary)
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int,
+@DefaultValue(0.05,0.25,0.5,0.75,0.95) @QueryParam(quantiles) 
quantileString: String
--- End diff --

I ran into a couple of limitations w/ `List[X]`.  (a) I found the longer 
URLs annoying `?quantile=0.01quantile=0.05quantile=0.25...` (b) I couldn't do 
nice error handling if parsing failed for individual `X`s (c) couldn't figure 
out how to specify a default with multiple values (which kinda makes sense, 
since its not clear what the behavior should be if the user added another value 
in that case).

(c) is pretty important in this case.  I am tempted to drop use of 
`List[X]` in the other places if we want consistency -- also I could put in 
better error handling.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27992855
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala ---
@@ -0,0 +1,192 @@
+/*
+ * 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.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.SparkException
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.StageUIData
+import org.apache.spark.util.SparkEnum
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneStageResource(uiRoot: UIRoot) {
+
+  @GET
+  @Path()
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int
+  ): Seq[StageData] = {
+forStage(appId, stageId){ (listener,stageAttempts) =
+  stageAttempts.map { case (status, stageInfo) =
+val stageUiData = listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId)).
+getOrElse(throw new SparkException(failed to get full stage 
data for stage:  +
+stageInfo.stageId + : + stageInfo.attemptId)
+)
+}
+AllStagesResource.stageUiToStageData(status, stageInfo, 
stageUiData,
+  includeDetails = true)
+  }
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+})
+  def oneAttemptData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int
+  ): StageData = {
+forStageAttempt(appId, stageId, attemptId) { case (status, stageInfo, 
stageUiData) =
+  AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData,
+includeDetails = true)
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+}/taskSummary)
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int,
+@DefaultValue(0.05,0.25,0.5,0.75,0.95) @QueryParam(quantiles) 
quantileString: String
+  ): TaskMetricDistributions = {
+forStageAttempt(appId, stageId, attemptId) { case (status, stageInfo, 
stageUiData) =
+  val quantiles = quantileString.split(,).map{s =
+try {
+  s.toDouble
+} catch {
+  case nfe: NumberFormatException =
+throw new BadParameterException(quantiles, double, s)
+}
+  }
+  
AllStagesResource.taskMetricDistributions(stageUiData.taskData.values, 
quantiles)
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+}/taskList)
+  def taskList(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int,
+@DefaultValue(0) @QueryParam(offset) offset: Int,
+@DefaultValue(20) @QueryParam(length) length: Int,
+@DefaultValue(ID) @QueryParam(sortBy) sortBy: TaskSorting
+  ): Seq[TaskData] = {
+forStageAttempt(appId, stageId, attemptId) { case (status, stageInfo, 
stageUiData) =
--- End diff --

good point.  I'll also rename `forStage` - `withStage`


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27977423
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala ---
@@ -0,0 +1,106 @@
+/*
+ * 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 java.util
+import java.util.Date
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllJobsResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[JobStatus]
+  ): Seq[JobData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val statusToJobs: Seq[(JobStatus, Seq[JobUIData])] =
+AllJobsResource.getStatusToJobs(ui)
+  val adjStatuses: util.List[JobStatus] = {
+if (statuses.isEmpty) {
+  java.util.Arrays.asList(JobStatus.values: _*)
+}
+else {
+  statuses
+}
+  }
+  val jobInfos = for {
+(status, jobs) - statusToJobs
+job - jobs if adjStatuses.contains(status)
+  } yield {
+AllJobsResource.convertJobData(job, ui.jobProgressListener, false)
+  }
+  jobInfos.sortBy{- _.jobId}
+}
+  }
+
+}
+
+private[v1] object AllJobsResource {
+
+  def getStatusToJobs(ui: SparkUI): Seq[(JobStatus, Seq[JobUIData])] = {
+val statusToJobs = ui.jobProgressListener.synchronized {
+  Seq(
+JobStatus.RUNNING - 
ui.jobProgressListener.activeJobs.values.toSeq,
--- End diff --

It does make a copy, though the scala docs are a little confusing on this.  
It will return a `scala.collection.Seq`, which in theory could be a 
`mutable.Seq`.  It actually always returns an `immutable.Seq`, however.  Even 
if you start with a `mutable.Seq` and call `.toSeq`, you get a copy in an 
`immutable.Seq`.  (If you start w/ an `immutable.Seq`, then you get back the 
same object, but that would be OK since its immutable in any case.)  Do you 
think thats OK, or should we do something more defensive here?

(You may be thinking of `Map.mapValues`, which is lazy)


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27980336
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala 
---
@@ -0,0 +1,39 @@
+/*
+* 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.ws.rs.{GET, PathParam, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.ui.exec.ExecutorsPage
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class ExecutorListResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String
+  ): Seq[ExecutorSummary] = {
+uiRoot.withSparkUI(appId) { ui =
+  val listener = ui.executorsListener
+  val storageStatusList = listener.storageStatusList
+  (0 until storageStatusList.size).map { statusId =
+ExecutorsPage.getExecInfo(listener, statusId)
--- End diff --

looks like it does not -- though neither does the current UI.  
(`JobProgressListener` does not implement `onExecutorRemoved`.)  I'll work on 
including it -- though I'm wondering if it should just be a separate PR?


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-91003106
  
  [Test build #29877 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29877/consoleFull)
 for   PR 4435 at commit 
[`d2ef58d`](https://github.com/apache/spark/commit/d2ef58d622ab60592eea453ea1cb292b369ddadb).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-91007271
  
  [Test build #29877 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29877/consoleFull)
 for   PR 4435 at commit 
[`d2ef58d`](https://github.com/apache/spark/commit/d2ef58d622ab60592eea453ea1cb292b369ddadb).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `abstract class MetricHelper[I,O](f: InternalTaskMetrics = 
Option[I]) `
  * `sealed abstract class TaskSorting extends SparkEnum `
  * `class ApplicationInfo(`
  * `class ExecutorStageSummary(`
  * `class ExecutorSummary(`
  * `class JobData(`
  * `class RDDStorageInfo(`
  * `class RDDDataDistribution(`
  * `class RDDPartitionInfo(`
  * `class StageData(`
  * `class TaskData(`
  * `class TaskMetrics(`
  * `class InputMetrics(`
  * `class OutputMetrics(`
  * `class ShuffleReadMetrics(`
  * `class ShuffleWriteMetrics(`
  * `class TaskMetricDistributions(`
  * `class InputMetricDistributions(`
  * `class OutputMetricDistributions(`
  * `class ShuffleReadMetricDistributions(`
  * `class ShuffleWriteMetricDistributions(`
  * `class AccumulableInfo (`
  * `throw new SparkException(It appears you are using SparkEnum 
in a class which does not  +`

 * This patch **removes the following dependencies:**
   * `RoaringBitmap-0.4.5.jar`
   * `activation-1.1.jar`
   * `akka-actor_2.10-2.3.4-spark.jar`
   * `akka-remote_2.10-2.3.4-spark.jar`
   * `akka-slf4j_2.10-2.3.4-spark.jar`
   * `aopalliance-1.0.jar`
   * `arpack_combined_all-0.1.jar`
   * `avro-1.7.7.jar`
   * `breeze-macros_2.10-0.11.2.jar`
   * `breeze_2.10-0.11.2.jar`
   * `chill-java-0.5.0.jar`
   * `chill_2.10-0.5.0.jar`
   * `commons-beanutils-1.7.0.jar`
   * `commons-beanutils-core-1.8.0.jar`
   * `commons-cli-1.2.jar`
   * `commons-codec-1.10.jar`
   * `commons-collections-3.2.1.jar`
   * `commons-compress-1.4.1.jar`
   * `commons-configuration-1.6.jar`
   * `commons-digester-1.8.jar`
   * `commons-httpclient-3.1.jar`
   * `commons-io-2.1.jar`
   * `commons-lang-2.5.jar`
   * `commons-lang3-3.3.2.jar`
   * `commons-math-2.1.jar`
   * `commons-math3-3.1.1.jar`
   * `commons-net-2.2.jar`
   * `compress-lzf-1.0.0.jar`
   * `config-1.2.1.jar`
   * `core-1.1.2.jar`
   * `curator-client-2.4.0.jar`
   * `curator-framework-2.4.0.jar`
   * `curator-recipes-2.4.0.jar`
   * `gmbal-api-only-3.0.0-b023.jar`
   * `grizzly-framework-2.1.2.jar`
   * `grizzly-http-2.1.2.jar`
   * `grizzly-http-server-2.1.2.jar`
   * `grizzly-http-servlet-2.1.2.jar`
   * `grizzly-rcm-2.1.2.jar`
   * `groovy-all-2.3.7.jar`
   * `guava-14.0.1.jar`
   * `guice-3.0.jar`
   * `hadoop-annotations-2.2.0.jar`
   * `hadoop-auth-2.2.0.jar`
   * `hadoop-client-2.2.0.jar`
   * `hadoop-common-2.2.0.jar`
   * `hadoop-hdfs-2.2.0.jar`
   * `hadoop-mapreduce-client-app-2.2.0.jar`
   * `hadoop-mapreduce-client-common-2.2.0.jar`
   * `hadoop-mapreduce-client-core-2.2.0.jar`
   * `hadoop-mapreduce-client-jobclient-2.2.0.jar`
   * `hadoop-mapreduce-client-shuffle-2.2.0.jar`
   * `hadoop-yarn-api-2.2.0.jar`
   * `hadoop-yarn-client-2.2.0.jar`
   * `hadoop-yarn-common-2.2.0.jar`
   * `hadoop-yarn-server-common-2.2.0.jar`
   * `ivy-2.4.0.jar`
   * `jackson-annotations-2.4.0.jar`
   * `jackson-core-2.4.4.jar`
   * `jackson-core-asl-1.8.8.jar`
   * `jackson-databind-2.4.4.jar`
   * `jackson-jaxrs-1.8.8.jar`
   * `jackson-mapper-asl-1.8.8.jar`
   * `jackson-module-scala_2.10-2.4.4.jar`
   * `jackson-xc-1.8.8.jar`
   * `jansi-1.4.jar`
   * `javax.inject-1.jar`
   * `javax.servlet-3.0.0.v201112011016.jar`
   * `javax.servlet-3.1.jar`
   * `javax.servlet-api-3.0.1.jar`
   * `jaxb-api-2.2.2.jar`
   * `jaxb-impl-2.2.3-1.jar`
   * `jcl-over-slf4j-1.7.10.jar`
   * `jersey-client-1.9.jar`
   * `jersey-core-1.9.jar`
   * `jersey-grizzly2-1.9.jar`
   * `jersey-guice-1.9.jar`
   * `jersey-json-1.9.jar`
   * `jersey-server-1.9.jar`
   * `jersey-test-framework-core-1.9.jar`
   * `jersey-test-framework-grizzly2-1.9.jar`
   * `jets3t-0.7.1.jar`
   * `jettison-1.1.jar`
   * `jetty-util-6.1.26.jar`
   * `jline-0.9.94.jar`
   * `jline-2.10.4.jar`
   * `jodd-core-3.6.3.jar`
   * `json4s-ast_2.10-3.2.10.jar`
   * `json4s-core_2.10-3.2.10.jar`
   * `json4s-jackson_2.10-3.2.10.jar`
   * `jsr305-1.3.9.jar`
   * `jtransforms-2.4.0.jar`
   * `jul-to-slf4j-1.7.10.jar`
   * `kryo-2.21.jar`
   * `log4j-1.2.17.jar`
   * `lz4-1.2.0.jar`
   

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-91007276
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29877/
Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-91010964
  
  [Test build #29881 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29881/consoleFull)
 for   PR 4435 at commit 
[`101a698`](https://github.com/apache/spark/commit/101a6981a3ce11de58b78e99c6e6061aefdaca36).


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27987986
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala ---
@@ -0,0 +1,160 @@
+/*
+ * 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
+
+@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}/jobs)
+  def getJobs(): AllJobsResource = {
+new AllJobsResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/jobs/{jobId: \\d+})
+  def getJob(): OneJobResource = {
+new OneJobResource(uiRoot)
+  }
+
+
+  @Path(applications/{appId}/executors)
+  def getExecutors(): ExecutorListResource = {
+new ExecutorListResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/stages)
+  def getStages(): AllStagesResource= {
+new AllStagesResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/stages/{stageId: \\d+})
+  def getStage(): OneStageResource= {
+new OneStageResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/storage/rdd)
+  def getRdds(): AllRDDResource = {
+new AllRDDResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/storage/rdd/{rddId: \\d+})
+  def getRdd(): OneRDDResource = {
+new OneRDDResource(uiRoot)
+  }
+
+}
+
+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: Seq[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)(f: SparkUI = T): T = {
+getSparkUI(appId) 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 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27993705
  
--- Diff: core/src/test/scala/org/apache/spark/JsonTestUtils.scala ---
@@ -0,0 +1,35 @@
+/*
+ * 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
+
+import org.json4s.JsonMethods
+import org.json4s._
+import org.json4s.jackson.JsonMethods
+
+trait JsonTestUtils {
+  def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) {
+val Diff(c, a, d) = validateJson diff expectedJson
--- End diff --

this is infix actually, but good point that it doesn't fit the style guide.

(Also made me realize that this refactoring was supposed to result in me 
deleting other cases this occured, so I'll update that as well)


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27994632
  
--- Diff: project/plugins.sbt ---
@@ -29,6 +29,8 @@ addSbtPlugin(com.eed3si9n % sbt-unidoc % 0.3.1)
 
 addSbtPlugin(com.cavorite % sbt-avro % 0.3.2)
 
+addSbtPlugin(io.spray % sbt-revolver % 0.7.2)
--- End diff --

no, this is definitely not necessary.  It was extremely useful to me during 
development, and figured it was a good addition in general.

This plugin lets you easily start  stop long-running processes (like the 
history server which never dies) within an sbt session, without having to kill 
 restart sbt.  You just run `re-start` to launch your main program, but it 
gives you the sbt shell back, so you can run more commands, and eventually run 
`re-stop`.

I figured it was harmless to drop in here as well, but I'll move to a 
separate PR.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-91016657
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29881/
Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-08 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4435#issuecomment-91016637
  
  [Test build #29881 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29881/consoleFull)
 for   PR 4435 at commit 
[`101a698`](https://github.com/apache/spark/commit/101a6981a3ce11de58b78e99c6e6061aefdaca36).
 * This patch **fails MiMa tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `abstract class MetricHelper[I,O](f: InternalTaskMetrics = 
Option[I]) `
  * `sealed abstract class TaskSorting extends SparkEnum `
  * `class ApplicationInfo(`
  * `class ExecutorStageSummary(`
  * `class ExecutorSummary(`
  * `class JobData(`
  * `class RDDStorageInfo(`
  * `class RDDDataDistribution(`
  * `class RDDPartitionInfo(`
  * `class StageData(`
  * `class TaskData(`
  * `class TaskMetrics(`
  * `class InputMetrics(`
  * `class OutputMetrics(`
  * `class ShuffleReadMetrics(`
  * `class ShuffleWriteMetrics(`
  * `class TaskMetricDistributions(`
  * `class InputMetricDistributions(`
  * `class OutputMetricDistributions(`
  * `class ShuffleReadMetricDistributions(`
  * `class ShuffleWriteMetricDistributions(`
  * `class AccumulableInfo (`
  * `throw new SparkException(It appears you are using SparkEnum 
in a class which does not  +`

 * This patch **removes the following dependencies:**
   * `RoaringBitmap-0.4.5.jar`
   * `activation-1.1.jar`
   * `akka-actor_2.10-2.3.4-spark.jar`
   * `akka-remote_2.10-2.3.4-spark.jar`
   * `akka-slf4j_2.10-2.3.4-spark.jar`
   * `aopalliance-1.0.jar`
   * `arpack_combined_all-0.1.jar`
   * `avro-1.7.7.jar`
   * `breeze-macros_2.10-0.11.2.jar`
   * `breeze_2.10-0.11.2.jar`
   * `chill-java-0.5.0.jar`
   * `chill_2.10-0.5.0.jar`
   * `commons-beanutils-1.7.0.jar`
   * `commons-beanutils-core-1.8.0.jar`
   * `commons-cli-1.2.jar`
   * `commons-codec-1.10.jar`
   * `commons-collections-3.2.1.jar`
   * `commons-compress-1.4.1.jar`
   * `commons-configuration-1.6.jar`
   * `commons-digester-1.8.jar`
   * `commons-httpclient-3.1.jar`
   * `commons-io-2.1.jar`
   * `commons-lang-2.5.jar`
   * `commons-lang3-3.3.2.jar`
   * `commons-math-2.1.jar`
   * `commons-math3-3.1.1.jar`
   * `commons-net-2.2.jar`
   * `compress-lzf-1.0.0.jar`
   * `config-1.2.1.jar`
   * `core-1.1.2.jar`
   * `curator-client-2.4.0.jar`
   * `curator-framework-2.4.0.jar`
   * `curator-recipes-2.4.0.jar`
   * `gmbal-api-only-3.0.0-b023.jar`
   * `grizzly-framework-2.1.2.jar`
   * `grizzly-http-2.1.2.jar`
   * `grizzly-http-server-2.1.2.jar`
   * `grizzly-http-servlet-2.1.2.jar`
   * `grizzly-rcm-2.1.2.jar`
   * `groovy-all-2.3.7.jar`
   * `guava-14.0.1.jar`
   * `guice-3.0.jar`
   * `hadoop-annotations-2.2.0.jar`
   * `hadoop-auth-2.2.0.jar`
   * `hadoop-client-2.2.0.jar`
   * `hadoop-common-2.2.0.jar`
   * `hadoop-hdfs-2.2.0.jar`
   * `hadoop-mapreduce-client-app-2.2.0.jar`
   * `hadoop-mapreduce-client-common-2.2.0.jar`
   * `hadoop-mapreduce-client-core-2.2.0.jar`
   * `hadoop-mapreduce-client-jobclient-2.2.0.jar`
   * `hadoop-mapreduce-client-shuffle-2.2.0.jar`
   * `hadoop-yarn-api-2.2.0.jar`
   * `hadoop-yarn-client-2.2.0.jar`
   * `hadoop-yarn-common-2.2.0.jar`
   * `hadoop-yarn-server-common-2.2.0.jar`
   * `ivy-2.4.0.jar`
   * `jackson-annotations-2.4.0.jar`
   * `jackson-core-2.4.4.jar`
   * `jackson-core-asl-1.8.8.jar`
   * `jackson-databind-2.4.4.jar`
   * `jackson-jaxrs-1.8.8.jar`
   * `jackson-mapper-asl-1.8.8.jar`
   * `jackson-module-scala_2.10-2.4.4.jar`
   * `jackson-xc-1.8.8.jar`
   * `jansi-1.4.jar`
   * `javax.inject-1.jar`
   * `javax.servlet-3.0.0.v201112011016.jar`
   * `javax.servlet-3.1.jar`
   * `javax.servlet-api-3.0.1.jar`
   * `jaxb-api-2.2.2.jar`
   * `jaxb-impl-2.2.3-1.jar`
   * `jcl-over-slf4j-1.7.10.jar`
   * `jersey-client-1.9.jar`
   * `jersey-core-1.9.jar`
   * `jersey-grizzly2-1.9.jar`
   * `jersey-guice-1.9.jar`
   * `jersey-json-1.9.jar`
   * `jersey-server-1.9.jar`
   * `jersey-test-framework-core-1.9.jar`
   * `jersey-test-framework-grizzly2-1.9.jar`
   * `jets3t-0.7.1.jar`
   * `jettison-1.1.jar`
   * `jetty-util-6.1.26.jar`
   * `jline-0.9.94.jar`
   * `jline-2.10.4.jar`
   * `jodd-core-3.6.3.jar`
   * `json4s-ast_2.10-3.2.10.jar`
   * `json4s-core_2.10-3.2.10.jar`
   * `json4s-jackson_2.10-3.2.10.jar`
   * `jsr305-1.3.9.jar`
   * `jtransforms-2.4.0.jar`
   * `jul-to-slf4j-1.7.10.jar`
   * `kryo-2.21.jar`
   * `log4j-1.2.17.jar`
   * `lz4-1.2.0.jar`
   * 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27908155
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala ---
@@ -0,0 +1,106 @@
+/*
+ * 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 java.util
+import java.util.Date
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllJobsResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[JobStatus]
+  ): Seq[JobData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val statusToJobs: Seq[(JobStatus, Seq[JobUIData])] =
+AllJobsResource.getStatusToJobs(ui)
+  val adjStatuses: util.List[JobStatus] = {
+if (statuses.isEmpty) {
+  java.util.Arrays.asList(JobStatus.values: _*)
+}
+else {
+  statuses
+}
+  }
+  val jobInfos = for {
+(status, jobs) - statusToJobs
+job - jobs if adjStatuses.contains(status)
+  } yield {
+AllJobsResource.convertJobData(job, ui.jobProgressListener, false)
+  }
+  jobInfos.sortBy{- _.jobId}
+}
+  }
+
+}
+
+private[v1] object AllJobsResource {
+
+  def getStatusToJobs(ui: SparkUI): Seq[(JobStatus, Seq[JobUIData])] = {
+val statusToJobs = ui.jobProgressListener.synchronized {
+  Seq(
+JobStatus.RUNNING - 
ui.jobProgressListener.activeJobs.values.toSeq,
--- End diff --

Does `toSeq` make a copy of the original values? Or is it lazily evaluated?

If the latter, the `synchronized` isn't helping much.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27907226
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
@@ -141,17 +139,24 @@ private[history] class FsHistoryProvider(conf: 
SparkConf) extends ApplicationHis
 }
   }
 
-  override def getListing(): Iterable[FsApplicationHistoryInfo] = 
applications.values
+  override def getListing(refresh: Boolean): 
Iterable[FsApplicationHistoryInfo] = {
+if (refresh) checkForLogs()
--- End diff --

nit:

if (blah) {
  code
}

We only use the shorthand when using `if..else` as a ternary.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27907503
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
@@ -141,17 +139,24 @@ private[history] class FsHistoryProvider(conf: 
SparkConf) extends ApplicationHis
 }
   }
 
-  override def getListing(): Iterable[FsApplicationHistoryInfo] = 
applications.values
+  override def getListing(refresh: Boolean): 
Iterable[FsApplicationHistoryInfo] = {
+if (refresh) checkForLogs()
+applications.values
+  }
 
   override def getAppUI(appId: String): Option[SparkUI] = {
 try {
-  applications.get(appId).map { info =
+  val appOpt = applications.get(appId).orElse {
--- End diff --

I think this deserves a comment or even a separate PR. It seems you're 
trying to force a refresh every time the user requests an app that is not 
found. That can get expensive and really racy, since `getListing()` is not 
exactly cheap and makes assumptions about being called from a single thread.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27908296
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala ---
@@ -0,0 +1,111 @@
+/*
+ * 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.ws.rs.{GET, PathParam, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageUtils}
+import org.apache.spark.ui.storage.StorageListener
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllRDDResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String
+  ): Seq[RDDStorageInfo] = {
+uiRoot.withSparkUI(appId) { ui =
+  // should all access on storageListener also be synchronized?
+  val storageStatusList = ui.storageListener.storageStatusList
+  val rddInfos = ui.storageListener.rddInfoList
+  rddInfos.map{rddInfo =
+AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, 
storageStatusList,
+  includeDetails = false)
+  }
+
+}
+  }
+
+}
+
+private[spark] object AllRDDResource {
+
+  def getRDDStorageInfo(
+  rddId: Int,
+  listener: StorageListener,
+  includeDetails: Boolean): Option[RDDStorageInfo] = {
+val storageStatusList = listener.storageStatusList
+listener.rddInfoList.find { _.id == rddId }.map { rddInfo =
+  getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails)
+}
+  }
+
+  def getRDDStorageInfo(
+rddId: Int,
+rddInfo: RDDInfo,
+storageStatusList: Seq[StorageStatus],
+includeDetails: Boolean
+  ): RDDStorageInfo = {
+val workers = storageStatusList.map { (rddId, _) }
+val blockLocations = StorageUtils.getRddBlockLocations(rddId, 
storageStatusList)
+val blocks = storageStatusList
+  .flatMap { _.rddBlocksById(rddId) }
+  .sortWith { _._1.name  _._1.name }
+  .map { case (blockId, status) =
+  (blockId, status, 
blockLocations.get(blockId).getOrElse(Seq[String](Unknown)))
--- End diff --

nit: indentation


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27907654
  
--- Diff: 
core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala ---
@@ -34,7 +34,7 @@ private[history] class HistoryPage(parent: HistoryServer) 
extends WebUIPage()
 val requestedIncomplete =
   
Option(request.getParameter(showIncomplete)).getOrElse(false).toBoolean
 
-val allApps = parent.getApplicationList().filter(_.completed != 
requestedIncomplete)
+val allApps = parent.getApplicationList(true).filter(_.completed != 
requestedIncomplete)
--- End diff --

Same comment. Forcing a refresh of the listing is expensive, and doesn't 
sound related to this PR.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27908469
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
@@ -0,0 +1,285 @@
+/*
+ * 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 java.util.Date
+import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.executor.{InputMetrics = InternalInputMetrics, 
OutputMetrics = InternalOutputMetrics, ShuffleReadMetrics = 
InternalShuffleReadMetrics, ShuffleWriteMetrics = InternalShuffleWriteMetrics, 
TaskMetrics = InternalTaskMetrics}
+import org.apache.spark.scheduler.{AccumulableInfo = 
InternalAccumulableInfo, StageInfo}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+import org.apache.spark.util.Distribution
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllStagesResource(uiRoot: UIRoot) {
+
+  @GET
+  def stageList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[StageStatus]
+  ): Seq[StageData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val listener = ui.jobProgressListener
+  val stageAndStatus = AllStagesResource.stagesAndStatus(ui)
+  val adjStatuses = {
+if (statuses.isEmpty()) {
+  java.util.Arrays.asList(StageStatus.values: _*)
+} else {
+  statuses
+}
+  }
+  for {
+(status, stageList) - stageAndStatus
+stageInfo: StageInfo - stageList if adjStatuses.contains(status)
+stageUiData: StageUIData - listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId))
+}
+  } yield {
+AllStagesResource.stageUiToStageData(status, stageInfo, 
stageUiData, includeDetails = false)
+  }
+}
+  }
+}
+
+private[v1] object AllStagesResource {
+  def stageUiToStageData(
+status: StageStatus,
+stageInfo: StageInfo,
+stageUiData: StageUIData,
+includeDetails: Boolean
+  ): StageData = {
+
+val taskData = if(includeDetails) {
+  Some(stageUiData.taskData.map { case (k, v) = k - 
convertTaskData(v) } )
+} else {
+  None
+}
+val executorSummary = if(includeDetails) {
--- End diff --

nit: space after `if` (also in other places)


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27913087
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala 
---
@@ -0,0 +1,39 @@
+/*
+* 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.ws.rs.{GET, PathParam, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.ui.exec.ExecutorsPage
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class ExecutorListResource(uiRoot: UIRoot) {
+
+  @GET
+  def jobsList(
+@PathParam(appId) appId: String
+  ): Seq[ExecutorSummary] = {
+uiRoot.withSparkUI(appId) { ui =
+  val listener = ui.executorsListener
+  val storageStatusList = listener.storageStatusList
+  (0 until storageStatusList.size).map { statusId =
+ExecutorsPage.getExecInfo(listener, statusId)
--- End diff --

Does this return valid information after an executor dies / is killed?


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27913556
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala ---
@@ -0,0 +1,160 @@
+/*
+ * 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
+
+@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}/jobs)
+  def getJobs(): AllJobsResource = {
+new AllJobsResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/jobs/{jobId: \\d+})
+  def getJob(): OneJobResource = {
+new OneJobResource(uiRoot)
+  }
+
+
+  @Path(applications/{appId}/executors)
+  def getExecutors(): ExecutorListResource = {
+new ExecutorListResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/stages)
+  def getStages(): AllStagesResource= {
+new AllStagesResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/stages/{stageId: \\d+})
+  def getStage(): OneStageResource= {
+new OneStageResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/storage/rdd)
+  def getRdds(): AllRDDResource = {
+new AllRDDResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/storage/rdd/{rddId: \\d+})
+  def getRdd(): OneRDDResource = {
+new OneRDDResource(uiRoot)
+  }
+
+}
+
+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: Seq[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)(f: SparkUI = T): T = {
+getSparkUI(appId) 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 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27913846
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala ---
@@ -0,0 +1,37 @@
+/*
+ * 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.ws.rs.{PathParam, GET, Produces}
+import javax.ws.rs.core.MediaType
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneRDDResource(uiRoot: UIRoot) {
+
+@GET
+def rddData(
+  @PathParam(appId) appId: String,
+  @PathParam(rddId) rddId: Int
+): RDDStorageInfo  = {
+  uiRoot.withSparkUI(appId) { ui =
+AllRDDResource.getRDDStorageInfo(rddId, ui.storageListener, 
true).getOrElse(
+  throw new IllegalArgumentException(no rdd found w/ id  + rddId)
--- End diff --

`NotFoundException`?


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27917309
  
--- Diff: 
core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala ---
@@ -14,22 +14,166 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.spark.deploy.history
 
-import javax.servlet.http.HttpServletRequest
-
-import scala.collection.mutable
+import java.io.{File, FileInputStream, FileWriter, IOException}
+import java.net.{HttpURLConnection, URL}
+import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
 
-import org.apache.hadoop.fs.Path
-import org.mockito.Mockito.{when}
-import org.scalatest.FunSuite
-import org.scalatest.Matchers
+import org.apache.commons.io.{FileUtils, IOUtils}
+import org.mockito.Mockito.when
+import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
 import org.scalatest.mock.MockitoSugar
 
+import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf}
 import org.apache.spark.ui.SparkUI
 
-class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar {
+class HistoryServerSuite extends FunSuite with BeforeAndAfter with 
Matchers with MockitoSugar
+  with JsonTestUtils {
+
+  private val logDir = new File(src/test/resources/spark-events)
+  private val expRoot = new 
File(src/test/resources/HistoryServerExpectations/)
+  private val port = 18080
--- End diff --

Unlikely to cause problems (fingers crossed?), but it's generally sketchy 
to hardcode port numbers in tests.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27917711
  
--- Diff: project/plugins.sbt ---
@@ -29,6 +29,8 @@ addSbtPlugin(com.eed3si9n % sbt-unidoc % 0.3.1)
 
 addSbtPlugin(com.cavorite % sbt-avro % 0.3.2)
 
+addSbtPlugin(io.spray % sbt-revolver % 0.7.2)
--- End diff --

Is this needed by this change? Feels like it could/should be its own PR if 
it's beneficial.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27914029
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala ---
@@ -0,0 +1,192 @@
+/*
+ * 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.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.SparkException
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.StageUIData
+import org.apache.spark.util.SparkEnum
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneStageResource(uiRoot: UIRoot) {
+
+  @GET
+  @Path()
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int
+  ): Seq[StageData] = {
+forStage(appId, stageId){ (listener,stageAttempts) =
+  stageAttempts.map { case (status, stageInfo) =
+val stageUiData = listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId)).
+getOrElse(throw new SparkException(failed to get full stage 
data for stage:  +
+stageInfo.stageId + : + stageInfo.attemptId)
+)
+}
+AllStagesResource.stageUiToStageData(status, stageInfo, 
stageUiData,
+  includeDetails = true)
+  }
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+})
+  def oneAttemptData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int
+  ): StageData = {
+forStageAttempt(appId, stageId, attemptId) { case (status, stageInfo, 
stageUiData) =
+  AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData,
+includeDetails = true)
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+}/taskSummary)
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int,
+@DefaultValue(0.05,0.25,0.5,0.75,0.95) @QueryParam(quantiles) 
quantileString: String
--- End diff --

I see in certain places you're using comma-separated values, while in 
others you're using `List[String]`. Is that intentional?


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27915470
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala ---
@@ -0,0 +1,38 @@
+/*
+ * 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.ws.rs.WebApplicationException
+import javax.ws.rs.core.Response
+
+import com.sun.jersey.spi.container.{ContainerRequest, 
ContainerRequestFilter}
+
+private[v1] class SecurityFilter extends ContainerRequestFilter with 
UIRootFromServletContext {
+  def filter(req: ContainerRequest): ContainerRequest = {
+val user = Option(req.getUserPrincipal).map { _.getName }.orNull
+if (uiRoot.securityManager.checkUIViewPermissions(user)) {
+  req
+} else {
+  throw new WebApplicationException(
+Response
+  .status(Response.Status.UNAUTHORIZED)
--- End diff --

I think `FORBIDDEN` is more correct here. You'd use `UNAUTHORIZED` if the 
user hasn't logged in yet.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27916000
  
--- Diff: core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala ---
@@ -32,28 +32,19 @@ private[ui] class RDDPage(parent: StorageTab) extends 
WebUIPage(rdd) {
   def render(request: HttpServletRequest): Seq[Node] = {
 val parameterId = request.getParameter(id)
 require(parameterId != null  parameterId.nonEmpty, Missing id 
parameter)
-
 val rddId = parameterId.toInt
-val storageStatusList = listener.storageStatusList
-val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse {
-  // Rather than crashing, render an RDD Not Found page
-  return UIUtils.headerSparkPage(RDD Not Found, Seq[Node](), parent)
+val rddStorageInfo = AllRDDResource.getRDDStorageInfo(rddId, listener,
--- End diff --

nit: indentation makes this hard to read. I'd suggest:

val rddStorageInfo = AllRDDResource
  .getRDDStorageInfo(rddId, listener, includeDetails = true)
  .getOrElse {
// Rather than crashing, render an RDD Not Found page
return UIUtils.headerSparkPage(RDD Not Found, Seq[Node](), parent)
  }



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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27912147
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
@@ -0,0 +1,285 @@
+/*
+ * 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 java.util.Date
+import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.executor.{InputMetrics = InternalInputMetrics, 
OutputMetrics = InternalOutputMetrics, ShuffleReadMetrics = 
InternalShuffleReadMetrics, ShuffleWriteMetrics = InternalShuffleWriteMetrics, 
TaskMetrics = InternalTaskMetrics}
+import org.apache.spark.scheduler.{AccumulableInfo = 
InternalAccumulableInfo, StageInfo}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+import org.apache.spark.util.Distribution
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllStagesResource(uiRoot: UIRoot) {
+
+  @GET
+  def stageList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[StageStatus]
+  ): Seq[StageData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val listener = ui.jobProgressListener
+  val stageAndStatus = AllStagesResource.stagesAndStatus(ui)
+  val adjStatuses = {
+if (statuses.isEmpty()) {
+  java.util.Arrays.asList(StageStatus.values: _*)
+} else {
+  statuses
+}
+  }
+  for {
+(status, stageList) - stageAndStatus
+stageInfo: StageInfo - stageList if adjStatuses.contains(status)
+stageUiData: StageUIData - listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId))
+}
+  } yield {
+AllStagesResource.stageUiToStageData(status, stageInfo, 
stageUiData, includeDetails = false)
+  }
+}
+  }
+}
+
+private[v1] object AllStagesResource {
+  def stageUiToStageData(
+status: StageStatus,
+stageInfo: StageInfo,
+stageUiData: StageUIData,
+includeDetails: Boolean
+  ): StageData = {
+
+val taskData = if(includeDetails) {
+  Some(stageUiData.taskData.map { case (k, v) = k - 
convertTaskData(v) } )
+} else {
+  None
+}
+val executorSummary = if(includeDetails) {
+  Some(stageUiData.executorSummary.map { case (k, summary) =
+k - new ExecutorStageSummary(
+  taskTime = summary.taskTime,
+  failedTasks = summary.failedTasks,
+  succeededTasks = summary.succeededTasks,
+  inputBytes = summary.inputBytes,
+  outputBytes = summary.outputBytes,
+  shuffleRead = summary.shuffleRead,
+  shuffleWrite = summary.shuffleWrite,
+  memoryBytesSpilled = summary.memoryBytesSpilled,
+  diskBytesSpilled = summary.diskBytesSpilled
+)
+  })
+} else {
+  None
+}
+
+val accumulableInfo = stageUiData.accumulables.values.map { 
convertAccumulableInfo }.toSeq
+
+new StageData(
+  status = status,
+  stageId = stageInfo.stageId,
+  attemptId = stageInfo.attemptId,
+  numActiveTasks = stageUiData.numActiveTasks,
+  numCompleteTasks = stageUiData.numCompleteTasks,
+  numFailedTasks = stageUiData.numFailedTasks,
+  executorRunTime = stageUiData.executorRunTime,
+  inputBytes = stageUiData.inputBytes,
+  inputRecords = stageUiData.inputRecords,
+  outputBytes = stageUiData.outputBytes,
+  outputRecords = stageUiData.outputRecords,
+  shuffleReadBytes = stageUiData.shuffleReadTotalBytes,
+  shuffleReadRecords = stageUiData.shuffleReadRecords,
+  shuffleWriteBytes = stageUiData.shuffleWriteBytes,
+  shuffleWriteRecords = 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27913682
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala ---
@@ -0,0 +1,160 @@
+/*
+ * 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
+
+@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}/jobs)
+  def getJobs(): AllJobsResource = {
+new AllJobsResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/jobs/{jobId: \\d+})
+  def getJob(): OneJobResource = {
+new OneJobResource(uiRoot)
+  }
+
+
+  @Path(applications/{appId}/executors)
+  def getExecutors(): ExecutorListResource = {
+new ExecutorListResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/stages)
+  def getStages(): AllStagesResource= {
+new AllStagesResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/stages/{stageId: \\d+})
+  def getStage(): OneStageResource= {
+new OneStageResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/storage/rdd)
+  def getRdds(): AllRDDResource = {
+new AllRDDResource(uiRoot)
+  }
+
+  @Path(applications/{appId}/storage/rdd/{rddId: \\d+})
+  def getRdd(): OneRDDResource = {
+new OneRDDResource(uiRoot)
+  }
+
+}
+
+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: Seq[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)(f: SparkUI = T): T = {
+getSparkUI(appId) 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 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27913939
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala ---
@@ -0,0 +1,192 @@
+/*
+ * 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.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.SparkException
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.StageUIData
+import org.apache.spark.util.SparkEnum
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneStageResource(uiRoot: UIRoot) {
+
+  @GET
+  @Path()
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int
+  ): Seq[StageData] = {
+forStage(appId, stageId){ (listener,stageAttempts) =
+  stageAttempts.map { case (status, stageInfo) =
+val stageUiData = listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId)).
+getOrElse(throw new SparkException(failed to get full stage 
data for stage:  +
--- End diff --

nit: move `.` to this line, indent next line.

If this the correct exception to throw here? It will show up as a 500 
error, most probably.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27915320
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala ---
@@ -0,0 +1,192 @@
+/*
+ * 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.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.SparkException
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.StageUIData
+import org.apache.spark.util.SparkEnum
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneStageResource(uiRoot: UIRoot) {
+
+  @GET
+  @Path()
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int
+  ): Seq[StageData] = {
+forStage(appId, stageId){ (listener,stageAttempts) =
+  stageAttempts.map { case (status, stageInfo) =
+val stageUiData = listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId)).
+getOrElse(throw new SparkException(failed to get full stage 
data for stage:  +
+stageInfo.stageId + : + stageInfo.attemptId)
+)
+}
+AllStagesResource.stageUiToStageData(status, stageInfo, 
stageUiData,
+  includeDetails = true)
+  }
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+})
+  def oneAttemptData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int
+  ): StageData = {
+forStageAttempt(appId, stageId, attemptId) { case (status, stageInfo, 
stageUiData) =
+  AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData,
+includeDetails = true)
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+}/taskSummary)
+  def stageData(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int,
+@DefaultValue(0.05,0.25,0.5,0.75,0.95) @QueryParam(quantiles) 
quantileString: String
+  ): TaskMetricDistributions = {
+forStageAttempt(appId, stageId, attemptId) { case (status, stageInfo, 
stageUiData) =
+  val quantiles = quantileString.split(,).map{s =
+try {
+  s.toDouble
+} catch {
+  case nfe: NumberFormatException =
+throw new BadParameterException(quantiles, double, s)
+}
+  }
+  
AllStagesResource.taskMetricDistributions(stageUiData.taskData.values, 
quantiles)
+}
+  }
+
+  @GET
+  @Path(/{attemptId: \\d+}/taskList)
+  def taskList(
+@PathParam(appId) appId: String,
+@PathParam(stageId) stageId: Int,
+@PathParam(attemptId) attemptId: Int,
+@DefaultValue(0) @QueryParam(offset) offset: Int,
+@DefaultValue(20) @QueryParam(length) length: Int,
+@DefaultValue(ID) @QueryParam(sortBy) sortBy: TaskSorting
+  ): Seq[TaskData] = {
+forStageAttempt(appId, stageId, attemptId) { case (status, stageInfo, 
stageUiData) =
+  val tasks = 
stageUiData.taskData.values.map{AllStagesResource.convertTaskData}.toIndexedSeq
+.sorted(sortBy.ordering)
+  tasks.slice(offset, offset + length)  
+}
+  }
+
+
+  def forStage[T](appId: String, stageId: Int)
+  (f: (JobProgressListener, Seq[(StageStatus, StageInfo)]) = T): T = {
+uiRoot.withSparkUI(appId) { ui =
+  val stageAndStatus = AllStagesResource.stagesAndStatus(ui)
+  val stageAttempts = stageAndStatus.flatMap { case (status, stages) =
+val matched = stages.filter { stage = stage.stageId == stageId}
+matched.map {
+  status - _
+}
+  }
+  if (stageAttempts.isEmpty) {
+throw new 

[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27915387
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala ---
@@ -0,0 +1,38 @@
+/*
+ * 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.ws.rs.WebApplicationException
+import javax.ws.rs.core.Response
+
+import com.sun.jersey.spi.container.{ContainerRequest, 
ContainerRequestFilter}
+
+private[v1] class SecurityFilter extends ContainerRequestFilter with 
UIRootFromServletContext {
+  def filter(req: ContainerRequest): ContainerRequest = {
+val user = Option(req.getUserPrincipal).map { _.getName }.orNull
+if (uiRoot.securityManager.checkUIViewPermissions(user)) {
+  req
+} else {
+  throw new WebApplicationException(
+Response
+  .status(Response.Status.UNAUTHORIZED)
+  .entity(user \ + user + \is not authorized)
--- End diff --

Use interpolation?


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27915782
  
--- Diff: core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala 
---
@@ -20,30 +20,13 @@ package org.apache.spark.ui.exec
 import java.net.URLEncoder
 import javax.servlet.http.HttpServletRequest
 
+import org.apache.spark.status.api.v1.ExecutorSummary
--- End diff --

nit: group with other Spark imports.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27916239
  
--- Diff: core/src/test/scala/org/apache/spark/JsonTestUtils.scala ---
@@ -0,0 +1,35 @@
+/*
+ * 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
--- End diff --

Funny. I wonder why github is not syntax-highlighting these files at the 
end of your PR.


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

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



[GitHub] spark pull request: [SPARK-3454] [WIP] separate json endpoints for...

2015-04-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4435#discussion_r27912432
  
--- Diff: 
core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
@@ -0,0 +1,285 @@
+/*
+ * 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 java.util.Date
+import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.executor.{InputMetrics = InternalInputMetrics, 
OutputMetrics = InternalOutputMetrics, ShuffleReadMetrics = 
InternalShuffleReadMetrics, ShuffleWriteMetrics = InternalShuffleWriteMetrics, 
TaskMetrics = InternalTaskMetrics}
+import org.apache.spark.scheduler.{AccumulableInfo = 
InternalAccumulableInfo, StageInfo}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+import org.apache.spark.util.Distribution
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllStagesResource(uiRoot: UIRoot) {
+
+  @GET
+  def stageList(
+@PathParam(appId) appId: String,
+@QueryParam(status) statuses: java.util.List[StageStatus]
+  ): Seq[StageData] = {
+uiRoot.withSparkUI(appId) { ui =
+  val listener = ui.jobProgressListener
+  val stageAndStatus = AllStagesResource.stagesAndStatus(ui)
+  val adjStatuses = {
+if (statuses.isEmpty()) {
+  java.util.Arrays.asList(StageStatus.values: _*)
+} else {
+  statuses
+}
+  }
+  for {
+(status, stageList) - stageAndStatus
+stageInfo: StageInfo - stageList if adjStatuses.contains(status)
+stageUiData: StageUIData - listener.synchronized {
+  listener.stageIdToData.get((stageInfo.stageId, 
stageInfo.attemptId))
+}
+  } yield {
+AllStagesResource.stageUiToStageData(status, stageInfo, 
stageUiData, includeDetails = false)
+  }
+}
+  }
+}
+
+private[v1] object AllStagesResource {
+  def stageUiToStageData(
+status: StageStatus,
+stageInfo: StageInfo,
+stageUiData: StageUIData,
+includeDetails: Boolean
+  ): StageData = {
+
+val taskData = if(includeDetails) {
+  Some(stageUiData.taskData.map { case (k, v) = k - 
convertTaskData(v) } )
+} else {
+  None
+}
+val executorSummary = if(includeDetails) {
+  Some(stageUiData.executorSummary.map { case (k, summary) =
+k - new ExecutorStageSummary(
+  taskTime = summary.taskTime,
+  failedTasks = summary.failedTasks,
+  succeededTasks = summary.succeededTasks,
+  inputBytes = summary.inputBytes,
+  outputBytes = summary.outputBytes,
+  shuffleRead = summary.shuffleRead,
+  shuffleWrite = summary.shuffleWrite,
+  memoryBytesSpilled = summary.memoryBytesSpilled,
+  diskBytesSpilled = summary.diskBytesSpilled
+)
+  })
+} else {
+  None
+}
+
+val accumulableInfo = stageUiData.accumulables.values.map { 
convertAccumulableInfo }.toSeq
+
+new StageData(
+  status = status,
+  stageId = stageInfo.stageId,
+  attemptId = stageInfo.attemptId,
+  numActiveTasks = stageUiData.numActiveTasks,
+  numCompleteTasks = stageUiData.numCompleteTasks,
+  numFailedTasks = stageUiData.numFailedTasks,
+  executorRunTime = stageUiData.executorRunTime,
+  inputBytes = stageUiData.inputBytes,
+  inputRecords = stageUiData.inputRecords,
+  outputBytes = stageUiData.outputBytes,
+  outputRecords = stageUiData.outputRecords,
+  shuffleReadBytes = stageUiData.shuffleReadTotalBytes,
+  shuffleReadRecords = stageUiData.shuffleReadRecords,
+  shuffleWriteBytes = stageUiData.shuffleWriteBytes,
+  shuffleWriteRecords = 

  1   2   >