1717
1818package org .apache .spark
1919
20- import scala .concurrent ._
21- import scala .concurrent .duration .Duration
22- import scala .util .Try
20+ import java .util .Collections
21+ import java .util .concurrent .TimeUnit
2322
24- import org .apache .spark .annotation . Experimental
23+ import org .apache .spark .api . java . JavaFutureAction
2524import org .apache .spark .rdd .RDD
2625import org .apache .spark .scheduler .{JobFailed , JobSucceeded , JobWaiter }
2726
27+ import scala .concurrent ._
28+ import scala .concurrent .duration .Duration
29+ import scala .util .{Failure , Try }
30+
2831/**
29- * :: Experimental ::
3032 * A future for the result of an action to support cancellation. This is an extension of the
3133 * Scala Future interface to support cancellation.
3234 */
33- @ Experimental
3435trait FutureAction [T ] extends Future [T ] {
3536 // Note that we redefine methods of the Future trait here explicitly so we can specify a different
3637 // documentation (with reference to the word "action").
@@ -69,6 +70,11 @@ trait FutureAction[T] extends Future[T] {
6970 */
7071 override def isCompleted : Boolean
7172
73+ /**
74+ * Returns whether the action has been cancelled.
75+ */
76+ def isCancelled : Boolean
77+
7278 /**
7379 * The value of this Future.
7480 *
@@ -96,15 +102,16 @@ trait FutureAction[T] extends Future[T] {
96102
97103
98104/**
99- * :: Experimental ::
100105 * A [[FutureAction ]] holding the result of an action that triggers a single job. Examples include
101106 * count, collect, reduce.
102107 */
103- @ Experimental
104108class SimpleFutureAction [T ] private [spark](jobWaiter : JobWaiter [_], resultFunc : => T )
105109 extends FutureAction [T ] {
106110
111+ @ volatile private var _cancelled : Boolean = false
112+
107113 override def cancel () {
114+ _cancelled = true
108115 jobWaiter.cancel()
109116 }
110117
@@ -143,6 +150,8 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc:
143150 }
144151
145152 override def isCompleted : Boolean = jobWaiter.jobFinished
153+
154+ override def isCancelled : Boolean = _cancelled
146155
147156 override def value : Option [Try [T ]] = {
148157 if (jobWaiter.jobFinished) {
@@ -164,12 +173,10 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc:
164173
165174
166175/**
167- * :: Experimental ::
168176 * A [[FutureAction ]] for actions that could trigger multiple Spark jobs. Examples include take,
169177 * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the
170178 * action thread if it is being blocked by a job.
171179 */
172- @ Experimental
173180class ComplexFutureAction [T ] extends FutureAction [T ] {
174181
175182 // Pointer to the thread that is executing the action. It is set when the action is run.
@@ -222,7 +229,7 @@ class ComplexFutureAction[T] extends FutureAction[T] {
222229 // If the action hasn't been cancelled yet, submit the job. The check and the submitJob
223230 // command need to be in an atomic block.
224231 val job = this .synchronized {
225- if (! cancelled ) {
232+ if (! isCancelled ) {
226233 rdd.context.submitJob(rdd, processPartition, partitions, resultHandler, resultFunc)
227234 } else {
228235 throw new SparkException (" Action has been cancelled" )
@@ -243,10 +250,7 @@ class ComplexFutureAction[T] extends FutureAction[T] {
243250 }
244251 }
245252
246- /**
247- * Returns whether the promise has been cancelled.
248- */
249- def cancelled : Boolean = _cancelled
253+ override def isCancelled : Boolean = _cancelled
250254
251255 @ throws(classOf [InterruptedException ])
252256 @ throws(classOf [scala.concurrent.TimeoutException ])
@@ -271,3 +275,55 @@ class ComplexFutureAction[T] extends FutureAction[T] {
271275 def jobIds = jobs
272276
273277}
278+
279+ private [spark]
280+ class JavaFutureActionWrapper [S , T ](futureAction : FutureAction [S ], converter : S => T )
281+ extends JavaFutureAction [T ] {
282+
283+ import scala .collection .JavaConverters ._
284+
285+ override def isCancelled : Boolean = futureAction.isCancelled
286+
287+ override def isDone : Boolean = {
288+ // According to java.util.Future's Javadoc, this returns True if the task was completed,
289+ // whether that completion was due to successful execution, an exception, or a cancellation.
290+ futureAction.isCancelled || futureAction.isCompleted
291+ }
292+
293+ override def jobIds (): java.util.List [java.lang.Integer ] = {
294+ Collections .unmodifiableList(futureAction.jobIds.map(Integer .valueOf).asJava)
295+ }
296+
297+ private def getImpl (timeout : Duration ): T = {
298+ // This will throw TimeoutException on timeout:
299+ Await .ready(futureAction, timeout)
300+ futureAction.value.get match {
301+ case scala.util.Success (value) => converter(value)
302+ case Failure (exception) =>
303+ if (isCancelled) {
304+ throw new CancellationException (" Job cancelled" ).initCause(exception)
305+ } else {
306+ // java.util.Future.get() wraps exceptions in ExecutionException
307+ throw new ExecutionException (" Exception thrown by job" , exception)
308+ }
309+ }
310+ }
311+
312+ override def get (): T = getImpl(Duration .Inf )
313+
314+ override def get (timeout : Long , unit : TimeUnit ): T =
315+ getImpl(Duration .fromNanos(unit.toNanos(timeout)))
316+
317+ override def cancel (mayInterruptIfRunning : Boolean ): Boolean = synchronized {
318+ if (isDone) {
319+ // According to java.util.Future's Javadoc, this should return false if the task is completed.
320+ false
321+ } else {
322+ // We're limited in terms of the semantics we can provide here; our cancellation is
323+ // asynchronous and doesn't provide a mechanism to not cancel if the job is running.
324+ futureAction.cancel()
325+ true
326+ }
327+ }
328+
329+ }
0 commit comments