diff --git a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala
index ba0d468f111efb2ae7d355fa0844acb868f86af6..0161962cde0736331577ed808dfe9160164e79d4 100644
--- a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala
@@ -29,9 +29,11 @@ import akka.actor.{ActorSystem, ExtendedActorSystem, Actor, ActorRef, Props, Add
 import akka.event.Logging.Error
 import akka.pattern.{ask => akkaAsk}
 import akka.remote.{AssociationEvent, AssociatedEvent, DisassociatedEvent, AssociationErrorEvent}
+import com.google.common.util.concurrent.MoreExecutors
+
 import org.apache.spark.{SparkException, Logging, SparkConf}
 import org.apache.spark.rpc._
-import org.apache.spark.util.{ActorLogReceive, AkkaUtils}
+import org.apache.spark.util.{ActorLogReceive, AkkaUtils, ThreadUtils}
 
 /**
  * A RpcEnv implementation based on Akka.
@@ -294,8 +296,8 @@ private[akka] class AkkaRpcEndpointRef(
   }
 
   override def ask[T: ClassTag](message: Any, timeout: FiniteDuration): Future[T] = {
-    import scala.concurrent.ExecutionContext.Implicits.global
     actorRef.ask(AkkaMessage(message, true))(timeout).flatMap {
+      // The function will run in the calling thread, so it should be short and never block.
       case msg @ AkkaMessage(message, reply) =>
         if (reply) {
           logError(s"Receive $msg but the sender cannot reply")
@@ -305,7 +307,7 @@ private[akka] class AkkaRpcEndpointRef(
         }
       case AkkaFailure(e) =>
         Future.failed(e)
-    }.mapTo[T]
+    }(ThreadUtils.sameThread).mapTo[T]
   }
 
   override def toString: String = s"${getClass.getSimpleName}($actorRef)"
diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
index 098a4b79496b201fe14f93ae28418d1a4fd2e470..ca5624a3d8b3d268bf641921b9bf2080bff542be 100644
--- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
@@ -20,10 +20,22 @@ package org.apache.spark.util
 
 import java.util.concurrent._
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutor}
+
+import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder}
 
 private[spark] object ThreadUtils {
 
+  private val sameThreadExecutionContext =
+    ExecutionContext.fromExecutorService(MoreExecutors.sameThreadExecutor())
+
+  /**
+   * An `ExecutionContextExecutor` that runs each task in the thread that invokes `execute/submit`.
+   * The caller should make sure the tasks running in this `ExecutionContextExecutor` are short and
+   * never block.
+   */
+  def sameThread: ExecutionContextExecutor = sameThreadExecutionContext
+
   /**
    * Create a thread factory that names threads with a prefix and also sets the threads to daemon.
    */
@@ -40,6 +52,16 @@ private[spark] object ThreadUtils {
     Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor]
   }
 
+  /**
+   * Create a cached thread pool whose max number of threads is `maxThreadNumber`. Thread names
+   * are formatted as prefix-ID, where ID is a unique, sequentially assigned integer.
+   */
+  def newDaemonCachedThreadPool(prefix: String, maxThreadNumber: Int): ThreadPoolExecutor = {
+    val threadFactory = namedThreadFactory(prefix)
+    new ThreadPoolExecutor(
+      0, maxThreadNumber, 60L, TimeUnit.SECONDS, new SynchronousQueue[Runnable], threadFactory)
+  }
+
   /**
    * Wrapper over newFixedThreadPool. Thread names are formatted as prefix-ID, where ID is a
    * unique, sequentially assigned integer.
diff --git a/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala
index a3aa3e953fbec64566db67fe56f766fb5c029cf4..751d3df9cc8f720fba1ba8f6fe2831a3c1bd6ad0 100644
--- a/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala
@@ -20,6 +20,9 @@ package org.apache.spark.util
 
 import java.util.concurrent.{CountDownLatch, TimeUnit}
 
+import scala.concurrent.{Await, Future}
+import scala.concurrent.duration._
+
 import org.scalatest.FunSuite
 
 class ThreadUtilsSuite extends FunSuite {
@@ -54,4 +57,13 @@ class ThreadUtilsSuite extends FunSuite {
       executor.shutdownNow()
     }
   }
+
+  test("sameThread") {
+    val callerThreadName = Thread.currentThread().getName()
+    val f = Future {
+      Thread.currentThread().getName()
+    }(ThreadUtils.sameThread)
+    val futureThreadName = Await.result(f, 10.seconds)
+    assert(futureThreadName === callerThreadName)
+  }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
index 05dd5681edfac21315b73b553c6cc272de787984..fe43fc4125c8e328d36aa69f5952116d24c668ee 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
@@ -18,10 +18,10 @@
 package org.apache.spark.sql.execution.joins
 
 import org.apache.spark.rdd.RDD
+import org.apache.spark.util.ThreadUtils
 
 import scala.concurrent._
 import scala.concurrent.duration._
-import scala.concurrent.ExecutionContext.Implicits.global
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql.catalyst.expressions.{Row, Expression}
@@ -64,7 +64,7 @@ case class BroadcastHashJoin(
     val input: Array[Row] = buildPlan.execute().map(_.copy()).collect()
     val hashed = HashedRelation(input.iterator, buildSideKeyGenerator, input.length)
     sparkContext.broadcast(hashed)
-  }
+  }(BroadcastHashJoin.broadcastHashJoinExecutionContext)
 
   protected override def doExecute(): RDD[Row] = {
     val broadcastRelation = Await.result(broadcastFuture, timeout)
@@ -74,3 +74,9 @@ case class BroadcastHashJoin(
     }
   }
 }
+
+object BroadcastHashJoin {
+
+  private val broadcastHashJoinExecutionContext = ExecutionContext.fromExecutorService(
+    ThreadUtils.newDaemonCachedThreadPool("broadcast-hash-join", 1024))
+}