aboutsummaryrefslogtreecommitdiff
path: root/core/src/test
diff options
context:
space:
mode:
authorzsxwing <zsxwing@gmail.com>2015-05-16 00:44:29 -0700
committerReynold Xin <rxin@databricks.com>2015-05-16 00:44:29 -0700
commit47e7ffe36b8a8a246fe9af522aff480d19c0c8a6 (patch)
tree8cfa6ec81fbd0f72e5c9bf74e8b1be0b5217b150 /core/src/test
parent0ac8b01a07840f199bbc79fb845762284aead6de (diff)
downloadspark-47e7ffe36b8a8a246fe9af522aff480d19c0c8a6.tar.gz
spark-47e7ffe36b8a8a246fe9af522aff480d19c0c8a6.tar.bz2
spark-47e7ffe36b8a8a246fe9af522aff480d19c0c8a6.zip
[SPARK-7655][Core][SQL] Remove 'scala.concurrent.ExecutionContext.Implicits.global' in 'ask' and 'BroadcastHashJoin'
Because both `AkkaRpcEndpointRef.ask` and `BroadcastHashJoin` uses `scala.concurrent.ExecutionContext.Implicits.global`. However, because the tasks in `BroadcastHashJoin` are usually long-running tasks, which will occupy all threads in `global`. Then `ask` cannot get a chance to process the replies. For `ask`, actually the tasks are very simple, so we can use `MoreExecutors.sameThreadExecutor()`. For `BroadcastHashJoin`, it's better to use `ThreadUtils.newDaemonCachedThreadPool`. Author: zsxwing <zsxwing@gmail.com> Closes #6200 from zsxwing/SPARK-7655-2 and squashes the following commits: cfdc605 [zsxwing] Remove redundant imort and minor doc fix cf83153 [zsxwing] Add "sameThread" and "newDaemonCachedThreadPool with maxThreadNumber" to ThreadUtils 08ad0ee [zsxwing] Remove 'scala.concurrent.ExecutionContext.Implicits.global' in 'ask' and 'BroadcastHashJoin'
Diffstat (limited to 'core/src/test')
-rw-r--r--core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala12
1 files changed, 12 insertions, 0 deletions
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 a3aa3e953f..751d3df9cc 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)
+ }
}