diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala
index 75ea535f2f57be20c4e7ec9289a8da5e306162d4..c277c3a47d421bb7e10055ddc3c4a555e6c70f12 100644
--- a/core/src/main/scala/org/apache/spark/FutureAction.scala
+++ b/core/src/main/scala/org/apache/spark/FutureAction.scala
@@ -208,7 +208,7 @@ class ComplexFutureAction[T] extends FutureAction[T] {
       processPartition: Iterator[T] => U,
       partitions: Seq[Int],
       resultHandler: (Int, U) => Unit,
-      resultFunc: => R) {
+      resultFunc: => R): R = {
     // If the action hasn't been cancelled yet, submit the job. The check and the submitJob
     // command need to be in an atomic block.
     val job = this.synchronized {
@@ -223,7 +223,10 @@ class ComplexFutureAction[T] extends FutureAction[T] {
     // 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.
     try {
-      Await.ready(job, Duration.Inf)
+      Await.ready(job, Duration.Inf).value.get match {
+        case scala.util.Failure(e) => throw e
+        case scala.util.Success(v) => v
+      }
     } catch {
       case e: InterruptedException =>
         job.cancel()
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index 37053bb6f37adfcb52c1b39e116d44ac65693fb3..d40b152d221c5f02c7a0921d9ae59bc8a1d01dff 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -29,6 +29,10 @@ import org.apache.spark.serializer.JavaSerializer
 import org.apache.spark.util.{CollectionsUtils, Utils}
 import org.apache.spark.util.random.{XORShiftRandom, SamplingUtils}
 
+import org.apache.spark.SparkContext.rddToAsyncRDDActions
+import scala.concurrent.Await
+import scala.concurrent.duration.Duration
+
 /**
  * An object that defines how the elements in a key-value pair RDD are partitioned by key.
  * Maps each key to a partition ID, from 0 to `numPartitions - 1`.
@@ -113,8 +117,12 @@ class RangePartitioner[K : Ordering : ClassTag, V](
   private var ordering = implicitly[Ordering[K]]
 
   // An array of upper bounds for the first (partitions - 1) partitions
-  private var rangeBounds: Array[K] = {
-    if (partitions <= 1) {
+  @volatile private var valRB: Array[K] = null
+
+  private def rangeBounds: Array[K] = this.synchronized {
+    if (valRB != null) return valRB
+
+    valRB = if (partitions <= 1) {
       Array.empty
     } else {
       // This is the sample size we need to have roughly balanced output partitions, capped at 1M.
@@ -152,6 +160,8 @@ class RangePartitioner[K : Ordering : ClassTag, V](
         RangePartitioner.determineBounds(candidates, partitions)
       }
     }
+
+    valRB
   }
 
   def numPartitions = rangeBounds.length + 1
@@ -222,7 +232,8 @@ class RangePartitioner[K : Ordering : ClassTag, V](
   }
 
   @throws(classOf[IOException])
-  private def readObject(in: ObjectInputStream) {
+  private def readObject(in: ObjectInputStream): Unit = this.synchronized {
+    if (valRB != null) return
     val sfactory = SparkEnv.get.serializer
     sfactory match {
       case js: JavaSerializer => in.defaultReadObject()
@@ -234,7 +245,7 @@ class RangePartitioner[K : Ordering : ClassTag, V](
         val ser = sfactory.newInstance()
         Utils.deserializeViaNestedStream(in, ser) { ds =>
           implicit val classTag = ds.readObject[ClassTag[Array[K]]]()
-          rangeBounds = ds.readObject[Array[K]]()
+          valRB = ds.readObject[Array[K]]()
         }
     }
   }
@@ -254,12 +265,18 @@ private[spark] object RangePartitioner {
       sampleSizePerPartition: Int): (Long, Array[(Int, Int, Array[K])]) = {
     val shift = rdd.id
     // val classTagK = classTag[K] // to avoid serializing the entire partitioner object
-    val sketched = rdd.mapPartitionsWithIndex { (idx, iter) =>
+    // use collectAsync here to run this job as a future, which is cancellable
+    val sketchFuture = rdd.mapPartitionsWithIndex { (idx, iter) =>
       val seed = byteswap32(idx ^ (shift << 16))
       val (sample, n) = SamplingUtils.reservoirSampleAndCount(
         iter, sampleSizePerPartition, seed)
       Iterator((idx, n, sample))
-    }.collect()
+    }.collectAsync()
+    // We do need the future's value to continue any further
+    val sketched = Await.ready(sketchFuture, Duration.Inf).value.get match {
+      case scala.util.Success(v) => v.toArray
+      case scala.util.Failure(e) => throw e
+    }
     val numItems = sketched.map(_._2.toLong).sum
     (numItems, sketched)
   }
diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
index b62f3fbdc4a15f946cedfe79b36351def5d3465d..7a68b3afa81581606a0614013fafc6f11c5b715d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer
 import scala.concurrent.ExecutionContext.Implicits.global
 import scala.reflect.ClassTag
 
+import org.apache.spark.util.Utils
 import org.apache.spark.{ComplexFutureAction, FutureAction, Logging}
 import org.apache.spark.annotation.Experimental
 
@@ -38,29 +39,30 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
    * Returns a future for counting the number of elements in the RDD.
    */
   def countAsync(): FutureAction[Long] = {
-    val totalCount = new AtomicLong
-    self.context.submitJob(
-      self,
-      (iter: Iterator[T]) => {
-        var result = 0L
-        while (iter.hasNext) {
-          result += 1L
-          iter.next()
-        }
-        result
-      },
-      Range(0, self.partitions.size),
-      (index: Int, data: Long) => totalCount.addAndGet(data),
-      totalCount.get())
+    val f = new ComplexFutureAction[Long]
+    f.run {
+      val totalCount = new AtomicLong
+      f.runJob(self,
+               (iter: Iterator[T]) => Utils.getIteratorSize(iter),
+               Range(0, self.partitions.size),
+               (index: Int, data: Long) => totalCount.addAndGet(data),
+               totalCount.get())
+    }
   }
 
   /**
    * Returns a future for retrieving all elements of this RDD.
    */
   def collectAsync(): FutureAction[Seq[T]] = {
-    val results = new Array[Array[T]](self.partitions.size)
-    self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.size),
-      (index, data) => results(index) = data, results.flatten.toSeq)
+    val f = new ComplexFutureAction[Seq[T]]
+    f.run {
+      val results = new Array[Array[T]](self.partitions.size)
+      f.runJob(self,
+               (iter: Iterator[T]) => iter.toArray,
+               Range(0, self.partitions.size),
+               (index: Int, data: Array[T]) => results(index) = data,
+               results.flatten.toSeq)
+    }
   }
 
   /**
@@ -104,24 +106,34 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
       }
       results.toSeq
     }
-
-    f
   }
 
   /**
    * Applies a function f to all elements of this RDD.
    */
-  def foreachAsync(f: T => Unit): FutureAction[Unit] = {
-    val cleanF = self.context.clean(f)
-    self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.size),
-      (index, data) => Unit, Unit)
+  def foreachAsync(expr: T => Unit): FutureAction[Unit] = {
+    val f = new ComplexFutureAction[Unit]
+    val exprClean = self.context.clean(expr)
+    f.run {
+      f.runJob(self,
+               (iter: Iterator[T]) => iter.foreach(exprClean),
+               Range(0, self.partitions.size),
+               (index: Int, data: Unit) => Unit,
+               Unit)
+    }
   }
 
   /**
    * Applies a function f to each partition of this RDD.
    */
-  def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = {
-    self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.size),
-      (index, data) => Unit, Unit)
+  def foreachPartitionAsync(expr: Iterator[T] => Unit): FutureAction[Unit] = {
+    val f = new ComplexFutureAction[Unit]
+    f.run {
+      f.runJob(self,
+               expr,
+               Range(0, self.partitions.size),
+               (index: Int, data: Unit) => Unit,
+               Unit)
+    }
   }
 }