You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by jo...@apache.org on 2014/10/02 04:25:14 UTC

git commit: [SPARK-3446] Expose underlying job ids in FutureAction.

Repository: spark
Updated Branches:
  refs/heads/master 93861a5e8 -> 29c351320


[SPARK-3446] Expose underlying job ids in FutureAction.

FutureAction is the only type exposed through the async APIs, so
for job IDs to be useful they need to be exposed there. The complication
is that some async jobs run more than one job (e.g. takeAsync),
so the exposed ID has to actually be a list of IDs that can actually
change over time. So the interface doesn't look very nice, but...

Change is actually small, I just added a basic test to make sure
it works.

Author: Marcelo Vanzin <va...@cloudera.com>

Closes #2337 from vanzin/SPARK-3446 and squashes the following commits:

e166a68 [Marcelo Vanzin] Fix comment.
1fed2bc [Marcelo Vanzin] [SPARK-3446] Expose underlying job ids in FutureAction.


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

Branch: refs/heads/master
Commit: 29c3513203218af33bea2f6d99d622cf263d55dd
Parents: 93861a5
Author: Marcelo Vanzin <va...@cloudera.com>
Authored: Wed Oct 1 19:24:22 2014 -0700
Committer: Josh Rosen <jo...@apache.org>
Committed: Wed Oct 1 19:24:22 2014 -0700

----------------------------------------------------------------------
 .../scala/org/apache/spark/FutureAction.scala   | 19 +++++++-
 .../org/apache/spark/FutureActionSuite.scala    | 49 ++++++++++++++++++++
 2 files changed, 66 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/29c35132/core/src/main/scala/org/apache/spark/FutureAction.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala
index 75ea535..e8f761e 100644
--- a/core/src/main/scala/org/apache/spark/FutureAction.scala
+++ b/core/src/main/scala/org/apache/spark/FutureAction.scala
@@ -83,6 +83,15 @@ trait FutureAction[T] extends Future[T] {
    */
   @throws(classOf[Exception])
   def get(): T = Await.result(this, Duration.Inf)
+
+  /**
+   * Returns the job IDs run by the underlying async operation.
+   *
+   * This returns the current snapshot of the job list. Certain operations may run multiple
+   * jobs, so multiple calls to this method may return different lists.
+   */
+  def jobIds: Seq[Int]
+
 }
 
 
@@ -150,8 +159,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc:
     }
   }
 
-  /** Get the corresponding job id for this action. */
-  def jobId = jobWaiter.jobId
+  def jobIds = Seq(jobWaiter.jobId)
 }
 
 
@@ -171,6 +179,8 @@ class ComplexFutureAction[T] extends FutureAction[T] {
   // is cancelled before the action was even run (and thus we have no thread to interrupt).
   @volatile private var _cancelled: Boolean = false
 
+  @volatile private var jobs: Seq[Int] = Nil
+
   // A promise used to signal the future.
   private val p = promise[T]()
 
@@ -219,6 +229,8 @@ class ComplexFutureAction[T] extends FutureAction[T] {
       }
     }
 
+    this.jobs = jobs ++ job.jobIds
+
     // Wait for the job to complete. If the action is cancelled (with an interrupt),
     // cancel the job and stop the execution. This is not in a synchronized block because
     // Await.ready eventually waits on the monitor in FutureJob.jobWaiter.
@@ -255,4 +267,7 @@ class ComplexFutureAction[T] extends FutureAction[T] {
   override def isCompleted: Boolean = p.isCompleted
 
   override def value: Option[Try[T]] = p.future.value
+
+  def jobIds = jobs
+
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/29c35132/core/src/test/scala/org/apache/spark/FutureActionSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala
new file mode 100644
index 0000000..db9c25f
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala
@@ -0,0 +1,49 @@
+/*
+ * 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.Await
+import scala.concurrent.duration.Duration
+
+import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
+
+import org.apache.spark.SparkContext._
+
+class FutureActionSuite extends FunSuite with BeforeAndAfter with Matchers with LocalSparkContext {
+
+  before {
+    sc = new SparkContext("local", "FutureActionSuite")
+  }
+
+  test("simple async action") {
+    val rdd = sc.parallelize(1 to 10, 2)
+    val job = rdd.countAsync()
+    val res = Await.result(job, Duration.Inf)
+    res should be (10)
+    job.jobIds.size should be (1)
+  }
+
+  test("complex async action") {
+    val rdd = sc.parallelize(1 to 15, 3)
+    val job = rdd.takeAsync(10)
+    val res = Await.result(job, Duration.Inf)
+    res should be (1 to 10)
+    job.jobIds.size should be (2)
+  }
+
+}


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