You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2014/11/15 08:46:35 UTC

spark git commit: [SPARK-2321] Several progress API improvements / refactorings

Repository: spark
Updated Branches:
  refs/heads/master cbddac236 -> 40eb8b6ef


[SPARK-2321] Several progress API improvements / refactorings

This PR refactors / extends the status API introduced in #2696.

- Change StatusAPI from a mixin trait to a class.  Before, the new status API methods were directly accessible through SparkContext, whereas now they're accessed through a `sc.statusAPI` field.  As long as we were going to add these methods directly to SparkContext, the mixin trait seemed like a good idea, but this might be simpler to reason about and may avoid pitfalls that I've run into while attempting to refactor other parts of SparkContext to use mixins (see #3071, for example).
- Change the name from SparkStatusAPI to SparkStatusTracker.
- Make `getJobIdsForGroup(null)` return ids for jobs that aren't associated with any job group.
- Add `getActiveStageIds()` and `getActiveJobIds()` methods that return the ids of whatever's currently active in this SparkContext.  This should simplify davies's progress bar code.

Author: Josh Rosen <jo...@databricks.com>

Closes #3197 from JoshRosen/progress-api-improvements and squashes the following commits:

30b0afa [Josh Rosen] Rename SparkStatusAPI to SparkStatusTracker.
d1b08d8 [Josh Rosen] Add missing newlines
2cc7353 [Josh Rosen] Add missing file.
d5eab1f [Josh Rosen] Add getActive[Stage|Job]Ids() methods.
a227984 [Josh Rosen] getJobIdsForGroup(null) should return jobs for default group
c47e294 [Josh Rosen] Remove StatusAPI mixin trait.


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

Branch: refs/heads/master
Commit: 40eb8b6ef3a67e36d0d9492c044981a1da76351d
Parents: cbddac2
Author: Josh Rosen <jo...@databricks.com>
Authored: Fri Nov 14 23:46:25 2014 -0800
Committer: Reynold Xin <rx...@databricks.com>
Committed: Fri Nov 14 23:46:25 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/SparkContext.scala   |  68 ++++++++-
 .../scala/org/apache/spark/SparkStatusAPI.scala | 142 -------------------
 .../org/apache/spark/SparkStatusTracker.scala   | 107 ++++++++++++++
 .../spark/api/java/JavaSparkContext.scala       |  21 +--
 .../spark/api/java/JavaSparkStatusTracker.scala |  72 ++++++++++
 .../scala/org/apache/spark/StatusAPISuite.scala |  78 ----------
 .../org/apache/spark/StatusTrackerSuite.scala   |  89 ++++++++++++
 .../spark/examples/JavaStatusAPIDemo.java       |  70 ---------
 .../spark/examples/JavaStatusTrackerDemo.java   |  70 +++++++++
 9 files changed, 407 insertions(+), 310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 03ea672..65edeef 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -25,6 +25,7 @@ import java.util.{Arrays, Properties, UUID}
 import java.util.concurrent.atomic.AtomicInteger
 import java.util.UUID.randomUUID
 import scala.collection.{Map, Set}
+import scala.collection.JavaConversions._
 import scala.collection.generic.Growable
 import scala.collection.mutable.HashMap
 import scala.reflect.{ClassTag, classTag}
@@ -61,7 +62,7 @@ import org.apache.spark.util._
  *   this config overrides the default configs as well as system properties.
  */
 
-class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
+class SparkContext(config: SparkConf) extends Logging {
 
   // This is used only by YARN for now, but should be relevant to other cluster types (Mesos,
   // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It
@@ -228,6 +229,8 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
   private[spark] val jobProgressListener = new JobProgressListener(conf)
   listenerBus.addListener(jobProgressListener)
 
+  val statusTracker = new SparkStatusTracker(this)
+
   // Initialize the Spark UI
   private[spark] val ui: Option[SparkUI] =
     if (conf.getBoolean("spark.ui.enabled", true)) {
@@ -1002,6 +1005,69 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
   def version = SPARK_VERSION
 
   /**
+   * Return a map from the slave to the max memory available for caching and the remaining
+   * memory available for caching.
+   */
+  def getExecutorMemoryStatus: Map[String, (Long, Long)] = {
+    env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) =>
+      (blockManagerId.host + ":" + blockManagerId.port, mem)
+    }
+  }
+
+  /**
+   * :: DeveloperApi ::
+   * Return information about what RDDs are cached, if they are in mem or on disk, how much space
+   * they take, etc.
+   */
+  @DeveloperApi
+  def getRDDStorageInfo: Array[RDDInfo] = {
+    val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray
+    StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus)
+    rddInfos.filter(_.isCached)
+  }
+
+  /**
+   * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call.
+   * Note that this does not necessarily mean the caching or computation was successful.
+   */
+  def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap
+
+  /**
+   * :: DeveloperApi ::
+   * Return information about blocks stored in all of the slaves
+   */
+  @DeveloperApi
+  def getExecutorStorageStatus: Array[StorageStatus] = {
+    env.blockManager.master.getStorageStatus
+  }
+
+  /**
+   * :: DeveloperApi ::
+   * Return pools for fair scheduler
+   */
+  @DeveloperApi
+  def getAllPools: Seq[Schedulable] = {
+    // TODO(xiajunluan): We should take nested pools into account
+    taskScheduler.rootPool.schedulableQueue.toSeq
+  }
+
+  /**
+   * :: DeveloperApi ::
+   * Return the pool associated with the given name, if one exists
+   */
+  @DeveloperApi
+  def getPoolForName(pool: String): Option[Schedulable] = {
+    Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool))
+  }
+
+  /**
+   * Return current scheduling mode
+   */
+  def getSchedulingMode: SchedulingMode.SchedulingMode = {
+    taskScheduler.schedulingMode
+  }
+
+  /**
    * Clear the job's list of files added by `addFile` so that they do not get downloaded to
    * any new nodes.
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala b/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala
deleted file mode 100644
index 1982499..0000000
--- a/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark
-
-import scala.collection.Map
-import scala.collection.JavaConversions._
-
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
-import org.apache.spark.scheduler.{SchedulingMode, Schedulable}
-import org.apache.spark.storage.{StorageStatus, StorageUtils, RDDInfo}
-
-/**
- * Trait that implements Spark's status APIs.  This trait is designed to be mixed into
- * SparkContext; it allows the status API code to live in its own file.
- */
-private[spark] trait SparkStatusAPI { this: SparkContext =>
-
-  /**
-   * Return a map from the slave to the max memory available for caching and the remaining
-   * memory available for caching.
-   */
-  def getExecutorMemoryStatus: Map[String, (Long, Long)] = {
-    env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) =>
-      (blockManagerId.host + ":" + blockManagerId.port, mem)
-    }
-  }
-
-  /**
-   * :: DeveloperApi ::
-   * Return information about what RDDs are cached, if they are in mem or on disk, how much space
-   * they take, etc.
-   */
-  @DeveloperApi
-  def getRDDStorageInfo: Array[RDDInfo] = {
-    val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray
-    StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus)
-    rddInfos.filter(_.isCached)
-  }
-
-  /**
-   * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call.
-   * Note that this does not necessarily mean the caching or computation was successful.
-   */
-  def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap
-
-  /**
-   * :: DeveloperApi ::
-   * Return information about blocks stored in all of the slaves
-   */
-  @DeveloperApi
-  def getExecutorStorageStatus: Array[StorageStatus] = {
-    env.blockManager.master.getStorageStatus
-  }
-
-  /**
-   * :: DeveloperApi ::
-   * Return pools for fair scheduler
-   */
-  @DeveloperApi
-  def getAllPools: Seq[Schedulable] = {
-    // TODO(xiajunluan): We should take nested pools into account
-    taskScheduler.rootPool.schedulableQueue.toSeq
-  }
-
-  /**
-   * :: DeveloperApi ::
-   * Return the pool associated with the given name, if one exists
-   */
-  @DeveloperApi
-  def getPoolForName(pool: String): Option[Schedulable] = {
-    Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool))
-  }
-
-  /**
-   * Return current scheduling mode
-   */
-  def getSchedulingMode: SchedulingMode.SchedulingMode = {
-    taskScheduler.schedulingMode
-  }
-
-
-  /**
-   * Return a list of all known jobs in a particular job group.  The returned list may contain
-   * running, failed, and completed jobs, and may vary across invocations of this method.  This
-   * method does not guarantee the order of the elements in its result.
-   */
-  def getJobIdsForGroup(jobGroup: String): Array[Int] = {
-    jobProgressListener.synchronized {
-      val jobData = jobProgressListener.jobIdToData.valuesIterator
-      jobData.filter(_.jobGroup.exists(_ == jobGroup)).map(_.jobId).toArray
-    }
-  }
-
-  /**
-   * Returns job information, or `None` if the job info could not be found or was garbage collected.
-   */
-  def getJobInfo(jobId: Int): Option[SparkJobInfo] = {
-    jobProgressListener.synchronized {
-      jobProgressListener.jobIdToData.get(jobId).map { data =>
-        new SparkJobInfoImpl(jobId, data.stageIds.toArray, data.status)
-      }
-    }
-  }
-
-  /**
-   * Returns stage information, or `None` if the stage info could not be found or was
-   * garbage collected.
-   */
-  def getStageInfo(stageId: Int): Option[SparkStageInfo] = {
-    jobProgressListener.synchronized {
-      for (
-        info <- jobProgressListener.stageIdToInfo.get(stageId);
-        data <- jobProgressListener.stageIdToData.get((stageId, info.attemptId))
-      ) yield {
-        new SparkStageInfoImpl(
-          stageId,
-          info.attemptId,
-          info.name,
-          info.numTasks,
-          data.numActiveTasks,
-          data.numCompleteTasks,
-          data.numFailedTasks)
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
new file mode 100644
index 0000000..c18d763
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.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
+
+/**
+ * Low-level status reporting APIs for monitoring job and stage progress.
+ *
+ * These APIs intentionally provide very weak consistency semantics; consumers of these APIs should
+ * be prepared to handle empty / missing information.  For example, a job's stage ids may be known
+ * but the status API may not have any information about the details of those stages, so
+ * `getStageInfo` could potentially return `None` for a valid stage id.
+ *
+ * To limit memory usage, these APIs only provide information on recent jobs / stages.  These APIs
+ * will provide information for the last `spark.ui.retainedStages` stages and
+ * `spark.ui.retainedJobs` jobs.
+ *
+ * NOTE: this class's constructor should be considered private and may be subject to change.
+ */
+class SparkStatusTracker private[spark] (sc: SparkContext) {
+
+  private val jobProgressListener = sc.jobProgressListener
+
+  /**
+   * Return a list of all known jobs in a particular job group.  If `jobGroup` is `null`, then
+   * returns all known jobs that are not associated with a job group.
+   *
+   * The returned list may contain running, failed, and completed jobs, and may vary across
+   * invocations of this method.  This method does not guarantee the order of the elements in
+   * its result.
+   */
+  def getJobIdsForGroup(jobGroup: String): Array[Int] = {
+    jobProgressListener.synchronized {
+      val jobData = jobProgressListener.jobIdToData.valuesIterator
+      jobData.filter(_.jobGroup.orNull == jobGroup).map(_.jobId).toArray
+    }
+  }
+
+  /**
+   * Returns an array containing the ids of all active stages.
+   *
+   * This method does not guarantee the order of the elements in its result.
+   */
+  def getActiveStageIds(): Array[Int] = {
+    jobProgressListener.synchronized {
+      jobProgressListener.activeStages.values.map(_.stageId).toArray
+    }
+  }
+
+  /**
+   * Returns an array containing the ids of all active jobs.
+   *
+   * This method does not guarantee the order of the elements in its result.
+   */
+  def getActiveJobIds(): Array[Int] = {
+    jobProgressListener.synchronized {
+      jobProgressListener.activeJobs.values.map(_.jobId).toArray
+    }
+  }
+
+  /**
+   * Returns job information, or `None` if the job info could not be found or was garbage collected.
+   */
+  def getJobInfo(jobId: Int): Option[SparkJobInfo] = {
+    jobProgressListener.synchronized {
+      jobProgressListener.jobIdToData.get(jobId).map { data =>
+        new SparkJobInfoImpl(jobId, data.stageIds.toArray, data.status)
+      }
+    }
+  }
+
+  /**
+   * Returns stage information, or `None` if the stage info could not be found or was
+   * garbage collected.
+   */
+  def getStageInfo(stageId: Int): Option[SparkStageInfo] = {
+    jobProgressListener.synchronized {
+      for (
+        info <- jobProgressListener.stageIdToInfo.get(stageId);
+        data <- jobProgressListener.stageIdToData.get((stageId, info.attemptId))
+      ) yield {
+        new SparkStageInfoImpl(
+          stageId,
+          info.attemptId,
+          info.name,
+          info.numTasks,
+          data.numActiveTasks,
+          data.numCompleteTasks,
+          data.numFailedTasks)
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 5c6e8d3..d50ed32 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -105,6 +105,8 @@ class JavaSparkContext(val sc: SparkContext)
 
   private[spark] val env = sc.env
 
+  def statusTracker = new JavaSparkStatusTracker(sc)
+
   def isLocal: java.lang.Boolean = sc.isLocal
 
   def sparkUser: String = sc.sparkUser
@@ -134,25 +136,6 @@ class JavaSparkContext(val sc: SparkContext)
   /** Default min number of partitions for Hadoop RDDs when not given by user */
   def defaultMinPartitions: java.lang.Integer = sc.defaultMinPartitions
 
-
-  /**
-   * Return a list of all known jobs in a particular job group.  The returned list may contain
-   * running, failed, and completed jobs, and may vary across invocations of this method.  This
-   * method does not guarantee the order of the elements in its result.
-   */
-  def getJobIdsForGroup(jobGroup: String): Array[Int] = sc.getJobIdsForGroup(jobGroup)
-
-  /**
-   * Returns job information, or `null` if the job info could not be found or was garbage collected.
-   */
-  def getJobInfo(jobId: Int): SparkJobInfo = sc.getJobInfo(jobId).orNull
-
-  /**
-   * Returns stage information, or `null` if the stage info could not be found or was
-   * garbage collected.
-   */
-  def getStageInfo(stageId: Int): SparkStageInfo = sc.getStageInfo(stageId).orNull
-
   /** Distribute a local Scala collection to form an RDD. */
   def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = {
     implicit val ctag: ClassTag[T] = fakeClassTag

http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/core/src/main/scala/org/apache/spark/api/java/JavaSparkStatusTracker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkStatusTracker.scala
new file mode 100644
index 0000000..3300cad
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkStatusTracker.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.api.java
+
+import org.apache.spark.{SparkStageInfo, SparkJobInfo, SparkContext}
+
+/**
+ * Low-level status reporting APIs for monitoring job and stage progress.
+ *
+ * These APIs intentionally provide very weak consistency semantics; consumers of these APIs should
+ * be prepared to handle empty / missing information.  For example, a job's stage ids may be known
+ * but the status API may not have any information about the details of those stages, so
+ * `getStageInfo` could potentially return `null` for a valid stage id.
+ *
+ * To limit memory usage, these APIs only provide information on recent jobs / stages.  These APIs
+ * will provide information for the last `spark.ui.retainedStages` stages and
+ * `spark.ui.retainedJobs` jobs.
+ *
+ * NOTE: this class's constructor should be considered private and may be subject to change.
+ */
+class JavaSparkStatusTracker private[spark] (sc: SparkContext) {
+
+  /**
+   * Return a list of all known jobs in a particular job group.  If `jobGroup` is `null`, then
+   * returns all known jobs that are not associated with a job group.
+   *
+   * The returned list may contain running, failed, and completed jobs, and may vary across
+   * invocations of this method.  This method does not guarantee the order of the elements in
+   * its result.
+   */
+  def getJobIdsForGroup(jobGroup: String): Array[Int] = sc.statusTracker.getJobIdsForGroup(jobGroup)
+
+  /**
+   * Returns an array containing the ids of all active stages.
+   *
+   * This method does not guarantee the order of the elements in its result.
+   */
+  def getActiveStageIds(): Array[Int] = sc.statusTracker.getActiveStageIds()
+
+  /**
+   * Returns an array containing the ids of all active jobs.
+   *
+   * This method does not guarantee the order of the elements in its result.
+   */
+  def getActiveJobIds(): Array[Int] = sc.statusTracker.getActiveJobIds()
+
+  /**
+   * Returns job information, or `null` if the job info could not be found or was garbage collected.
+   */
+  def getJobInfo(jobId: Int): SparkJobInfo = sc.statusTracker.getJobInfo(jobId).orNull
+
+  /**
+   * Returns stage information, or `null` if the stage info could not be found or was
+   * garbage collected.
+   */
+  def getStageInfo(stageId: Int): SparkStageInfo = sc.statusTracker.getStageInfo(stageId).orNull
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/core/src/test/scala/org/apache/spark/StatusAPISuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/StatusAPISuite.scala b/core/src/test/scala/org/apache/spark/StatusAPISuite.scala
deleted file mode 100644
index 4468fba..0000000
--- a/core/src/test/scala/org/apache/spark/StatusAPISuite.scala
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark
-
-import scala.concurrent.duration._
-import scala.language.implicitConversions
-import scala.language.postfixOps
-
-import org.scalatest.{Matchers, FunSuite}
-import org.scalatest.concurrent.Eventually._
-
-import org.apache.spark.JobExecutionStatus._
-import org.apache.spark.SparkContext._
-
-class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext {
-
-  test("basic status API usage") {
-    val jobFuture = sc.parallelize(1 to 10000, 2).map(identity).groupBy(identity).collectAsync()
-    val jobId: Int = eventually(timeout(10 seconds)) {
-      val jobIds = jobFuture.jobIds
-      jobIds.size should be(1)
-      jobIds.head
-    }
-    val jobInfo = eventually(timeout(10 seconds)) {
-      sc.getJobInfo(jobId).get
-    }
-    jobInfo.status() should not be FAILED
-    val stageIds = jobInfo.stageIds()
-    stageIds.size should be(2)
-
-    val firstStageInfo = eventually(timeout(10 seconds)) {
-      sc.getStageInfo(stageIds(0)).get
-    }
-    firstStageInfo.stageId() should be(stageIds(0))
-    firstStageInfo.currentAttemptId() should be(0)
-    firstStageInfo.numTasks() should be(2)
-    eventually(timeout(10 seconds)) {
-      val updatedFirstStageInfo = sc.getStageInfo(stageIds(0)).get
-      updatedFirstStageInfo.numCompletedTasks() should be(2)
-      updatedFirstStageInfo.numActiveTasks() should be(0)
-      updatedFirstStageInfo.numFailedTasks() should be(0)
-    }
-  }
-
-  test("getJobIdsForGroup()") {
-    sc.setJobGroup("my-job-group", "description")
-    sc.getJobIdsForGroup("my-job-group") should be (Seq.empty)
-    val firstJobFuture = sc.parallelize(1 to 1000).countAsync()
-    val firstJobId = eventually(timeout(10 seconds)) {
-      firstJobFuture.jobIds.head
-    }
-    eventually(timeout(10 seconds)) {
-      sc.getJobIdsForGroup("my-job-group") should be (Seq(firstJobId))
-    }
-    val secondJobFuture = sc.parallelize(1 to 1000).countAsync()
-    val secondJobId = eventually(timeout(10 seconds)) {
-      secondJobFuture.jobIds.head
-    }
-    eventually(timeout(10 seconds)) {
-      sc.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId))
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
new file mode 100644
index 0000000..8577e4a
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
@@ -0,0 +1,89 @@
+/*
+ * 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 scala.concurrent.duration._
+import scala.language.implicitConversions
+import scala.language.postfixOps
+
+import org.scalatest.{Matchers, FunSuite}
+import org.scalatest.concurrent.Eventually._
+
+import org.apache.spark.JobExecutionStatus._
+import org.apache.spark.SparkContext._
+
+class StatusTrackerSuite extends FunSuite with Matchers with LocalSparkContext {
+
+  test("basic status API usage") {
+    sc = new SparkContext("local", "test", new SparkConf(false))
+    val jobFuture = sc.parallelize(1 to 10000, 2).map(identity).groupBy(identity).collectAsync()
+    val jobId: Int = eventually(timeout(10 seconds)) {
+      val jobIds = jobFuture.jobIds
+      jobIds.size should be(1)
+      jobIds.head
+    }
+    val jobInfo = eventually(timeout(10 seconds)) {
+      sc.statusTracker.getJobInfo(jobId).get
+    }
+    jobInfo.status() should not be FAILED
+    val stageIds = jobInfo.stageIds()
+    stageIds.size should be(2)
+
+    val firstStageInfo = eventually(timeout(10 seconds)) {
+      sc.statusTracker.getStageInfo(stageIds(0)).get
+    }
+    firstStageInfo.stageId() should be(stageIds(0))
+    firstStageInfo.currentAttemptId() should be(0)
+    firstStageInfo.numTasks() should be(2)
+    eventually(timeout(10 seconds)) {
+      val updatedFirstStageInfo = sc.statusTracker.getStageInfo(stageIds(0)).get
+      updatedFirstStageInfo.numCompletedTasks() should be(2)
+      updatedFirstStageInfo.numActiveTasks() should be(0)
+      updatedFirstStageInfo.numFailedTasks() should be(0)
+    }
+  }
+
+  test("getJobIdsForGroup()") {
+    sc = new SparkContext("local", "test", new SparkConf(false))
+    // Passing `null` should return jobs that were not run in a job group:
+    val defaultJobGroupFuture = sc.parallelize(1 to 1000).countAsync()
+    val defaultJobGroupJobId = eventually(timeout(10 seconds)) {
+      defaultJobGroupFuture.jobIds.head
+    }
+    eventually(timeout(10 seconds)) {
+      sc.statusTracker.getJobIdsForGroup(null).toSet should be (Set(defaultJobGroupJobId))
+    }
+    // Test jobs submitted in job groups:
+    sc.setJobGroup("my-job-group", "description")
+    sc.statusTracker.getJobIdsForGroup("my-job-group") should be (Seq.empty)
+    val firstJobFuture = sc.parallelize(1 to 1000).countAsync()
+    val firstJobId = eventually(timeout(10 seconds)) {
+      firstJobFuture.jobIds.head
+    }
+    eventually(timeout(10 seconds)) {
+      sc.statusTracker.getJobIdsForGroup("my-job-group") should be (Seq(firstJobId))
+    }
+    val secondJobFuture = sc.parallelize(1 to 1000).countAsync()
+    val secondJobId = eventually(timeout(10 seconds)) {
+      secondJobFuture.jobIds.head
+    }
+    eventually(timeout(10 seconds)) {
+      sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId))
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java b/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java
deleted file mode 100644
index 430e96a..0000000
--- a/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.examples;
-
-import org.apache.spark.SparkConf;
-import org.apache.spark.SparkJobInfo;
-import org.apache.spark.SparkStageInfo;
-import org.apache.spark.api.java.JavaFutureAction;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.Function;
-
-import java.util.Arrays;
-import java.util.List;
-
-/**
- * Example of using Spark's status APIs from Java.
- */
-public final class JavaStatusAPIDemo {
-
-  public static final String APP_NAME = "JavaStatusAPIDemo";
-
-  public static final class IdentityWithDelay<T> implements Function<T, T> {
-    @Override
-    public T call(T x) throws Exception {
-      Thread.sleep(2 * 1000);  // 2 seconds
-      return x;
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    SparkConf sparkConf = new SparkConf().setAppName(APP_NAME);
-    final JavaSparkContext sc = new JavaSparkContext(sparkConf);
-
-    // Example of implementing a progress reporter for a simple job.
-    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map(
-        new IdentityWithDelay<Integer>());
-    JavaFutureAction<List<Integer>> jobFuture = rdd.collectAsync();
-    while (!jobFuture.isDone()) {
-      Thread.sleep(1000);  // 1 second
-      List<Integer> jobIds = jobFuture.jobIds();
-      if (jobIds.isEmpty()) {
-        continue;
-      }
-      int currentJobId = jobIds.get(jobIds.size() - 1);
-      SparkJobInfo jobInfo = sc.getJobInfo(currentJobId);
-      SparkStageInfo stageInfo = sc.getStageInfo(jobInfo.stageIds()[0]);
-      System.out.println(stageInfo.numTasks() + " tasks total: " + stageInfo.numActiveTasks() +
-          " active, " + stageInfo.numCompletedTasks() + " complete");
-    }
-
-    System.out.println("Job results are: " + jobFuture.get());
-    sc.stop();
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/40eb8b6e/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java
new file mode 100644
index 0000000..e68ec74
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java
@@ -0,0 +1,70 @@
+/*
+ * 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.examples;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.SparkJobInfo;
+import org.apache.spark.SparkStageInfo;
+import org.apache.spark.api.java.JavaFutureAction;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Example of using Spark's status APIs from Java.
+ */
+public final class JavaStatusTrackerDemo {
+
+  public static final String APP_NAME = "JavaStatusAPIDemo";
+
+  public static final class IdentityWithDelay<T> implements Function<T, T> {
+    @Override
+    public T call(T x) throws Exception {
+      Thread.sleep(2 * 1000);  // 2 seconds
+      return x;
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    SparkConf sparkConf = new SparkConf().setAppName(APP_NAME);
+    final JavaSparkContext sc = new JavaSparkContext(sparkConf);
+
+    // Example of implementing a progress reporter for a simple job.
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map(
+        new IdentityWithDelay<Integer>());
+    JavaFutureAction<List<Integer>> jobFuture = rdd.collectAsync();
+    while (!jobFuture.isDone()) {
+      Thread.sleep(1000);  // 1 second
+      List<Integer> jobIds = jobFuture.jobIds();
+      if (jobIds.isEmpty()) {
+        continue;
+      }
+      int currentJobId = jobIds.get(jobIds.size() - 1);
+      SparkJobInfo jobInfo = sc.statusTracker().getJobInfo(currentJobId);
+      SparkStageInfo stageInfo = sc.statusTracker().getStageInfo(jobInfo.stageIds()[0]);
+      System.out.println(stageInfo.numTasks() + " tasks total: " + stageInfo.numActiveTasks() +
+          " active, " + stageInfo.numCompletedTasks() + " complete");
+    }
+
+    System.out.println("Job results are: " + jobFuture.get());
+    sc.stop();
+  }
+}


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