Skip to content

Commit 5a7af9e

Browse files
nonglidavies
authored andcommitted
[SPARK-13250] [SQL] Update PhysicallRDD to convert to UnsafeRow if using the vectorized scanner.
Some parts of the engine rely on UnsafeRow which the vectorized parquet scanner does not want to produce. This add a conversion in Physical RDD. In the case where codegen is used (and the scan is the start of the pipeline), there is no requirement to use UnsafeRow. This patch adds update PhysicallRDD to support codegen, which eliminates the need for the UnsafeRow conversion in all cases. The result of these changes for TPCDS-Q19 at the 10gb sf reduces the query time from 9.5 seconds to 6.5 seconds. Author: Nong Li <[email protected]> Closes #11141 from nongli/spark-13250.
1 parent cbb0b65 commit 5a7af9e

File tree

7 files changed

+129
-71
lines changed

7 files changed

+129
-71
lines changed

python/pyspark/sql/dataframe.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -173,7 +173,8 @@ def explain(self, extended=False):
173173
174174
>>> df.explain()
175175
== Physical Plan ==
176-
Scan ExistingRDD[age#0,name#1]
176+
WholeStageCodegen
177+
: +- Scan ExistingRDD[age#0,name#1]
177178
178179
>>> df.explain(True)
179180
== Parsed Logical Plan ==

sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala

Lines changed: 41 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,14 @@
1818
package org.apache.spark.sql.execution
1919

2020
import org.apache.spark.rdd.RDD
21-
import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
21+
import org.apache.spark.sql.{AnalysisException, Row, SQLConf, SQLContext}
2222
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
2323
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
2424
import org.apache.spark.sql.catalyst.expressions._
25+
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
2526
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
2627
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
28+
import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
2729
import org.apache.spark.sql.execution.metric.SQLMetrics
2830
import org.apache.spark.sql.sources.{BaseRelation, HadoopFsRelation}
2931
import org.apache.spark.sql.types.DataType
@@ -102,7 +104,7 @@ private[sql] case class PhysicalRDD(
102104
override val metadata: Map[String, String] = Map.empty,
103105
isUnsafeRow: Boolean = false,
104106
override val outputPartitioning: Partitioning = UnknownPartitioning(0))
105-
extends LeafNode {
107+
extends LeafNode with CodegenSupport {
106108

107109
private[sql] override lazy val metrics = Map(
108110
"numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
@@ -128,6 +130,36 @@ private[sql] case class PhysicalRDD(
128130
val metadataEntries = for ((key, value) <- metadata.toSeq.sorted) yield s"$key: $value"
129131
s"Scan $nodeName${output.mkString("[", ",", "]")}${metadataEntries.mkString(" ", ", ", "")}"
130132
}
133+
134+
override def upstreams(): Seq[RDD[InternalRow]] = {
135+
rdd :: Nil
136+
}
137+
138+
// Support codegen so that we can avoid the UnsafeRow conversion in all cases. Codegen
139+
// never requires UnsafeRow as input.
140+
override protected def doProduce(ctx: CodegenContext): String = {
141+
val input = ctx.freshName("input")
142+
// PhysicalRDD always just has one input
143+
ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
144+
145+
val exprs = output.zipWithIndex.map(x => new BoundReference(x._2, x._1.dataType, true))
146+
val row = ctx.freshName("row")
147+
val numOutputRows = metricTerm(ctx, "numOutputRows")
148+
ctx.INPUT_ROW = row
149+
ctx.currentVars = null
150+
val columns = exprs.map(_.gen(ctx))
151+
s"""
152+
| while ($input.hasNext()) {
153+
| InternalRow $row = (InternalRow) $input.next();
154+
| $numOutputRows.add(1);
155+
| ${columns.map(_.code).mkString("\n").trim}
156+
| ${consume(ctx, columns).trim}
157+
| if (shouldStop()) {
158+
| return;
159+
| }
160+
| }
161+
""".stripMargin
162+
}
131163
}
132164

133165
private[sql] object PhysicalRDD {
@@ -140,8 +172,13 @@ private[sql] object PhysicalRDD {
140172
rdd: RDD[InternalRow],
141173
relation: BaseRelation,
142174
metadata: Map[String, String] = Map.empty): PhysicalRDD = {
143-
// All HadoopFsRelations output UnsafeRows
144-
val outputUnsafeRows = relation.isInstanceOf[HadoopFsRelation]
175+
val outputUnsafeRows = if (relation.isInstanceOf[ParquetRelation]) {
176+
// The vectorized parquet reader does not produce unsafe rows.
177+
!SQLContext.getActive().get.conf.getConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED)
178+
} else {
179+
// All HadoopFsRelations output UnsafeRows
180+
relation.isInstanceOf[HadoopFsRelation]
181+
}
145182

146183
val bucketSpec = relation match {
147184
case r: HadoopFsRelation => r.getBucketSpec

sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,7 @@ trait CodegenSupport extends SparkPlan {
4242
case _: TungstenAggregate => "agg"
4343
case _: BroadcastHashJoin => "bhj"
4444
case _: SortMergeJoin => "smj"
45+
case _: PhysicalRDD => "rdd"
4546
case _ => nodeName.toLowerCase
4647
}
4748

sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -187,8 +187,10 @@ class JDBCSuite extends SparkFunSuite
187187
val parentPlan = df.queryExecution.executedPlan
188188
// Check if SparkPlan Filter is removed in a physical plan and
189189
// the plan only has PhysicalRDD to scan JDBCRelation.
190-
assert(parentPlan.isInstanceOf[PhysicalRDD])
191-
assert(parentPlan.asInstanceOf[PhysicalRDD].nodeName.contains("JDBCRelation"))
190+
assert(parentPlan.isInstanceOf[org.apache.spark.sql.execution.WholeStageCodegen])
191+
val node = parentPlan.asInstanceOf[org.apache.spark.sql.execution.WholeStageCodegen]
192+
assert(node.plan.isInstanceOf[org.apache.spark.sql.execution.PhysicalRDD])
193+
assert(node.plan.asInstanceOf[PhysicalRDD].nodeName.contains("JDBCRelation"))
192194
df
193195
}
194196
assert(checkPushdown(sql("SELECT * FROM foobar WHERE THEID < 1")).collect().size == 0)

sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala

Lines changed: 31 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -304,30 +304,38 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic
304304
expectedCount: Int,
305305
requiredColumnNames: Set[String],
306306
expectedUnhandledFilters: Set[Filter]): Unit = {
307+
307308
test(s"PushDown Returns $expectedCount: $sqlString") {
308-
val queryExecution = sql(sqlString).queryExecution
309-
val rawPlan = queryExecution.executedPlan.collect {
310-
case p: execution.PhysicalRDD => p
311-
} match {
312-
case Seq(p) => p
313-
case _ => fail(s"More than one PhysicalRDD found\n$queryExecution")
314-
}
315-
val rawCount = rawPlan.execute().count()
316-
assert(ColumnsRequired.set === requiredColumnNames)
317-
318-
val table = caseInsensitiveContext.table("oneToTenFiltered")
319-
val relation = table.queryExecution.logical.collectFirst {
320-
case LogicalRelation(r, _, _) => r
321-
}.get
322-
323-
assert(
324-
relation.unhandledFilters(FiltersPushed.list.toArray).toSet === expectedUnhandledFilters)
325-
326-
if (rawCount != expectedCount) {
327-
fail(
328-
s"Wrong # of results for pushed filter. Got $rawCount, Expected $expectedCount\n" +
329-
s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" +
330-
queryExecution)
309+
// These tests check a particular plan, disable whole stage codegen.
310+
caseInsensitiveContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, false)
311+
try {
312+
val queryExecution = sql(sqlString).queryExecution
313+
val rawPlan = queryExecution.executedPlan.collect {
314+
case p: execution.PhysicalRDD => p
315+
} match {
316+
case Seq(p) => p
317+
case _ => fail(s"More than one PhysicalRDD found\n$queryExecution")
318+
}
319+
val rawCount = rawPlan.execute().count()
320+
assert(ColumnsRequired.set === requiredColumnNames)
321+
322+
val table = caseInsensitiveContext.table("oneToTenFiltered")
323+
val relation = table.queryExecution.logical.collectFirst {
324+
case LogicalRelation(r, _, _) => r
325+
}.get
326+
327+
assert(
328+
relation.unhandledFilters(FiltersPushed.list.toArray).toSet === expectedUnhandledFilters)
329+
330+
if (rawCount != expectedCount) {
331+
fail(
332+
s"Wrong # of results for pushed filter. Got $rawCount, Expected $expectedCount\n" +
333+
s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" +
334+
queryExecution)
335+
}
336+
} finally {
337+
caseInsensitiveContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED,
338+
SQLConf.WHOLESTAGE_CODEGEN_ENABLED.defaultValue.get)
331339
}
332340
}
333341
}

sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala

Lines changed: 26 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -117,28 +117,35 @@ class PrunedScanSuite extends DataSourceTest with SharedSQLContext {
117117

118118
def testPruning(sqlString: String, expectedColumns: String*): Unit = {
119119
test(s"Columns output ${expectedColumns.mkString(",")}: $sqlString") {
120-
val queryExecution = sql(sqlString).queryExecution
121-
val rawPlan = queryExecution.executedPlan.collect {
122-
case p: execution.PhysicalRDD => p
123-
} match {
124-
case Seq(p) => p
125-
case _ => fail(s"More than one PhysicalRDD found\n$queryExecution")
126-
}
127-
val rawColumns = rawPlan.output.map(_.name)
128-
val rawOutput = rawPlan.execute().first()
129-
130-
if (rawColumns != expectedColumns) {
131-
fail(
132-
s"Wrong column names. Got $rawColumns, Expected $expectedColumns\n" +
133-
s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" +
134-
queryExecution)
135-
}
136120

137-
if (rawOutput.numFields != expectedColumns.size) {
138-
fail(s"Wrong output row. Got $rawOutput\n$queryExecution")
121+
// These tests check a particular plan, disable whole stage codegen.
122+
caseInsensitiveContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, false)
123+
try {
124+
val queryExecution = sql(sqlString).queryExecution
125+
val rawPlan = queryExecution.executedPlan.collect {
126+
case p: execution.PhysicalRDD => p
127+
} match {
128+
case Seq(p) => p
129+
case _ => fail(s"More than one PhysicalRDD found\n$queryExecution")
130+
}
131+
val rawColumns = rawPlan.output.map(_.name)
132+
val rawOutput = rawPlan.execute().first()
133+
134+
if (rawColumns != expectedColumns) {
135+
fail(
136+
s"Wrong column names. Got $rawColumns, Expected $expectedColumns\n" +
137+
s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" +
138+
queryExecution)
139+
}
140+
141+
if (rawOutput.numFields != expectedColumns.size) {
142+
fail(s"Wrong output row. Got $rawOutput\n$queryExecution")
143+
}
144+
} finally {
145+
caseInsensitiveContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED,
146+
SQLConf.WHOLESTAGE_CODEGEN_ENABLED.defaultValue.get)
139147
}
140148
}
141149
}
142-
143150
}
144151

sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala

Lines changed: 24 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -74,32 +74,34 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet
7474
bucketValues: Seq[Integer],
7575
filterCondition: Column,
7676
originalDataFrame: DataFrame): Unit = {
77+
// This test verifies parts of the plan. Disable whole stage codegen.
78+
withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") {
79+
val bucketedDataFrame = hiveContext.table("bucketed_table").select("i", "j", "k")
80+
val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec
81+
// Limit: bucket pruning only works when the bucket column has one and only one column
82+
assert(bucketColumnNames.length == 1)
83+
val bucketColumnIndex = bucketedDataFrame.schema.fieldIndex(bucketColumnNames.head)
84+
val bucketColumn = bucketedDataFrame.schema.toAttributes(bucketColumnIndex)
85+
val matchedBuckets = new BitSet(numBuckets)
86+
bucketValues.foreach { value =>
87+
matchedBuckets.set(DataSourceStrategy.getBucketId(bucketColumn, numBuckets, value))
88+
}
7789

78-
val bucketedDataFrame = hiveContext.table("bucketed_table").select("i", "j", "k")
79-
val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec
80-
// Limit: bucket pruning only works when the bucket column has one and only one column
81-
assert(bucketColumnNames.length == 1)
82-
val bucketColumnIndex = bucketedDataFrame.schema.fieldIndex(bucketColumnNames.head)
83-
val bucketColumn = bucketedDataFrame.schema.toAttributes(bucketColumnIndex)
84-
val matchedBuckets = new BitSet(numBuckets)
85-
bucketValues.foreach { value =>
86-
matchedBuckets.set(DataSourceStrategy.getBucketId(bucketColumn, numBuckets, value))
87-
}
90+
// Filter could hide the bug in bucket pruning. Thus, skipping all the filters
91+
val plan = bucketedDataFrame.filter(filterCondition).queryExecution.executedPlan
92+
val rdd = plan.find(_.isInstanceOf[PhysicalRDD])
93+
assert(rdd.isDefined, plan)
8894

89-
// Filter could hide the bug in bucket pruning. Thus, skipping all the filters
90-
val rdd = bucketedDataFrame.filter(filterCondition).queryExecution.executedPlan
91-
.find(_.isInstanceOf[PhysicalRDD])
92-
assert(rdd.isDefined)
95+
val checkedResult = rdd.get.execute().mapPartitionsWithIndex { case (index, iter) =>
96+
if (matchedBuckets.get(index % numBuckets)) Iterator(true) else Iterator(iter.isEmpty)
97+
}
98+
// checking if all the pruned buckets are empty
99+
assert(checkedResult.collect().forall(_ == true))
93100

94-
val checkedResult = rdd.get.execute().mapPartitionsWithIndex { case (index, iter) =>
95-
if (matchedBuckets.get(index % numBuckets)) Iterator(true) else Iterator(iter.isEmpty)
101+
checkAnswer(
102+
bucketedDataFrame.filter(filterCondition).orderBy("i", "j", "k"),
103+
originalDataFrame.filter(filterCondition).orderBy("i", "j", "k"))
96104
}
97-
// checking if all the pruned buckets are empty
98-
assert(checkedResult.collect().forall(_ == true))
99-
100-
checkAnswer(
101-
bucketedDataFrame.filter(filterCondition).orderBy("i", "j", "k"),
102-
originalDataFrame.filter(filterCondition).orderBy("i", "j", "k"))
103105
}
104106

105107
test("read partitioning bucketed tables with bucket pruning filters") {

0 commit comments

Comments
 (0)