Skip to content

Commit 6ea5055

Browse files
rdbluesrowen
authored andcommitted
[SPARK-17396][CORE] Share the task support between UnionRDD instances.
## What changes were proposed in this pull request? Share the ForkJoinTaskSupport between UnionRDD instances to avoid creating a huge number of threads if lots of RDDs are created at the same time. ## How was this patch tested? This uses existing UnionRDD tests. Author: Ryan Blue <blue@apache.org> Closes #14985 from rdblue/SPARK-17396-use-shared-pool.
1 parent bcdd259 commit 6ea5055

File tree

1 file changed

+7
-5
lines changed

1 file changed

+7
-5
lines changed

core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.rdd
2020
import java.io.{IOException, ObjectOutputStream}
2121

2222
import scala.collection.mutable.ArrayBuffer
23-
import scala.collection.parallel.ForkJoinTaskSupport
23+
import scala.collection.parallel.{ForkJoinTaskSupport, ThreadPoolTaskSupport}
2424
import scala.concurrent.forkjoin.ForkJoinPool
2525
import scala.reflect.ClassTag
2626

@@ -58,6 +58,11 @@ private[spark] class UnionPartition[T: ClassTag](
5858
}
5959
}
6060

61+
object UnionRDD {
62+
private[spark] lazy val partitionEvalTaskSupport =
63+
new ForkJoinTaskSupport(new ForkJoinPool(8))
64+
}
65+
6166
@DeveloperApi
6267
class UnionRDD[T: ClassTag](
6368
sc: SparkContext,
@@ -68,13 +73,10 @@ class UnionRDD[T: ClassTag](
6873
private[spark] val isPartitionListingParallel: Boolean =
6974
rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10)
7075

71-
@transient private lazy val partitionEvalTaskSupport =
72-
new ForkJoinTaskSupport(new ForkJoinPool(8))
73-
7476
override def getPartitions: Array[Partition] = {
7577
val parRDDs = if (isPartitionListingParallel) {
7678
val parArray = rdds.par
77-
parArray.tasksupport = partitionEvalTaskSupport
79+
parArray.tasksupport = UnionRDD.partitionEvalTaskSupport
7880
parArray
7981
} else {
8082
rdds

0 commit comments

Comments
 (0)