Skip to content

Commit

Permalink
[SQL] When creating partitioned table scan, explicitly create UnionRDD.
Browse files Browse the repository at this point in the history
Otherwise, it will cause stack overflow when there are many partitions.

Author: Yin Huai <yhuai@databricks.com>

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

fa016d8 [Yin Huai] Explicitly create UnionRDD.
  • Loading branch information
yhuai authored and liancheng committed May 15, 2015
1 parent f9705d4 commit e8f0e01
Showing 1 changed file with 7 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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)
}
Expand Down

0 comments on commit e8f0e01

Please sign in to comment.