Skip to content
Snippets Groups Projects
Commit e8f0e016 authored by Yin Huai's avatar Yin Huai Committed by Cheng Lian
Browse files

[SQL] When creating partitioned table scan, explicitly create UnionRDD.

Otherwise, it will cause stack overflow when there are many partitions.

Author: Yin Huai <yhuai@databricks.com>

Closes #6162 from yhuai/partitionUnionedRDD and squashes the following commits:

fa016d8 [Yin Huai] Explicitly create UnionRDD.
parent f9705d46
No related branches found
No related tags found
No related merge requests found
......@@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.Logging
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.{UnionRDD, RDD}
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.expressions._
......@@ -169,9 +169,12 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
scan.execute()
}
val unionedRows = perPartitionRows.reduceOption(_ ++ _).getOrElse {
relation.sqlContext.emptyResult
}
val unionedRows =
if (perPartitionRows.length == 0) {
relation.sqlContext.emptyResult
} else {
new UnionRDD(relation.sqlContext.sparkContext, perPartitionRows)
}
createPhysicalRDD(logicalRelation.relation, output, unionedRows)
}
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment