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