Github user dbtsai commented on a diff in the pull request:
https://github.com/apache/spark/pull/11242#discussion_r56719728
--- Diff: core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala ---
@@ -62,7 +64,23 @@ class UnionRDD[T: ClassTag](
var rdds: Seq[RDD[T]])
extends RDD[T](sc, Nil) { // Nil since we implement getDependencies
+ // Evaluate partitions in parallel. Partitions of each rdd will be
cached by the `partitions`
+ // val in `RDD`.
+ private[spark] lazy val parallelPartitionEval: Boolean = {
+ val threshold = conf.getInt("spark.rdd.parallelListingThreshold", 10)
+ if (threshold > 0 && rdds.length > threshold) {
+ val parArray = rdds.toParArray
+ parArray.tasksupport = new ForkJoinTaskSupport(
+ new ForkJoinPool(math.min(threshold, 8)))
--- End diff --
In this case, it will always be 8 threads by default. How about
```scala
private[spark] lazy val parallelPartitionEval: Boolean = {
val parallelism = conf.getInt("spark.rdd.listingParallelism", 16)
val threshold = 8
if (rdds.length > threshold) {
val parArray = rdds.toParArray
parArray.tasksupport = new ForkJoinTaskSupport(
new ForkJoinPool(math.min(parallelism, rdds.length)))
parArray.foreach(_.partitions)
true
} else {
false
}
}
```
Thanks.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]