Skip to content

Commit e8f0e01

Browse files
yhuailiancheng
authored andcommitted
[SQL] When creating partitioned table scan, explicitly create UnionRDD.
Otherwise, it will cause stack overflow when there are many partitions. Author: Yin Huai <[email protected]> Closes apache#6162 from yhuai/partitionUnionedRDD and squashes the following commits: fa016d8 [Yin Huai] Explicitly create UnionRDD.
1 parent f9705d4 commit e8f0e01

File tree

1 file changed

+7
-4
lines changed

1 file changed

+7
-4
lines changed

sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path
2121

2222
import org.apache.spark.Logging
2323
import org.apache.spark.deploy.SparkHadoopUtil
24-
import org.apache.spark.rdd.RDD
24+
import org.apache.spark.rdd.{UnionRDD, RDD}
2525
import org.apache.spark.sql.Row
2626
import org.apache.spark.sql.catalyst.expressions
2727
import org.apache.spark.sql.catalyst.expressions._
@@ -169,9 +169,12 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
169169
scan.execute()
170170
}
171171

172-
val unionedRows = perPartitionRows.reduceOption(_ ++ _).getOrElse {
173-
relation.sqlContext.emptyResult
174-
}
172+
val unionedRows =
173+
if (perPartitionRows.length == 0) {
174+
relation.sqlContext.emptyResult
175+
} else {
176+
new UnionRDD(relation.sqlContext.sparkContext, perPartitionRows)
177+
}
175178

176179
createPhysicalRDD(logicalRelation.relation, output, unionedRows)
177180
}

0 commit comments

Comments
 (0)