aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorJosh Rosen <joshrosen@databricks.com>2014-11-14 23:46:25 -0800
committerReynold Xin <rxin@databricks.com>2014-11-14 23:46:25 -0800
commit40eb8b6ef3a67e36d0d9492c044981a1da76351d (patch)
tree9138527b5d90b13ad2195e7d34661db5f2c9d2d5 /core
parentcbddac23696d89b672dce380cc7360a873e27b3b (diff)
downloadspark-40eb8b6ef3a67e36d0d9492c044981a1da76351d.tar.gz
spark-40eb8b6ef3a67e36d0d9492c044981a1da76351d.tar.bz2
spark-40eb8b6ef3a67e36d0d9492c044981a1da76351d.zip
[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 <joshrosen@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.
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala68
-rw-r--r--core/src/main/scala/org/apache/spark/SparkStatusAPI.scala142
-rw-r--r--core/src/main/scala/org/apache/spark/SparkStatusTracker.scala107
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala21
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkStatusTracker.scala72
-rw-r--r--core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala (renamed from core/src/test/scala/org/apache/spark/StatusAPISuite.scala)25
6 files changed, 266 insertions, 169 deletions
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 03ea672c81..65edeeffb8 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.
*/
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 1982499c5e..0000000000
--- 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)
- }
- }
- }
-}
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 0000000000..c18d763d7f
--- /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)
+ }
+ }
+ }
+}
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 5c6e8d32c5..d50ed32ca0 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
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 0000000000..3300cad9ef
--- /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
+}
diff --git a/core/src/test/scala/org/apache/spark/StatusAPISuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
index 4468fba8c1..8577e4ac7e 100644
--- a/core/src/test/scala/org/apache/spark/StatusAPISuite.scala
+++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
@@ -27,9 +27,10 @@ import org.scalatest.concurrent.Eventually._
import org.apache.spark.JobExecutionStatus._
import org.apache.spark.SparkContext._
-class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext {
+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
@@ -37,20 +38,20 @@ class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext {
jobIds.head
}
val jobInfo = eventually(timeout(10 seconds)) {
- sc.getJobInfo(jobId).get
+ 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.getStageInfo(stageIds(0)).get
+ 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.getStageInfo(stageIds(0)).get
+ val updatedFirstStageInfo = sc.statusTracker.getStageInfo(stageIds(0)).get
updatedFirstStageInfo.numCompletedTasks() should be(2)
updatedFirstStageInfo.numActiveTasks() should be(0)
updatedFirstStageInfo.numFailedTasks() should be(0)
@@ -58,21 +59,31 @@ class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext {
}
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.getJobIdsForGroup("my-job-group") should be (Seq.empty)
+ 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.getJobIdsForGroup("my-job-group") should be (Seq(firstJobId))
+ 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.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId))
+ sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId))
}
}
} \ No newline at end of file