Skip to content
Snippets Groups Projects
Commit 39a77e15 authored by Davies Liu's avatar Davies Liu Committed by Davies Liu
Browse files

[SPARK-14856] [SQL] returning batch correctly

## What changes were proposed in this pull request?

Currently, the Parquet reader decide whether to return batch based on required schema or full schema, it's not consistent, this PR fix that.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #12619 from davies/fix_return_batch.
parent c0611018
No related branches found
No related tags found
No related merge requests found
...@@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => Parq ...@@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => Parq
import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types.DataType import org.apache.spark.sql.types.{DataType, StructType}
object RDDConversions { object RDDConversions {
def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = { def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = {
...@@ -348,7 +348,8 @@ private[sql] object DataSourceScanExec { ...@@ -348,7 +348,8 @@ private[sql] object DataSourceScanExec {
} }
relation match { relation match {
case r: HadoopFsRelation if r.fileFormat.supportBatch(r.sqlContext, relation.schema) => case r: HadoopFsRelation
if r.fileFormat.supportBatch(r.sqlContext, StructType.fromAttributes(output)) =>
BatchedDataSourceScanExec(output, rdd, relation, outputPartitioning, metadata) BatchedDataSourceScanExec(output, rdd, relation, outputPartitioning, metadata)
case _ => case _ =>
RowDataSourceScanExec(output, rdd, relation, outputPartitioning, metadata) RowDataSourceScanExec(output, rdd, relation, outputPartitioning, metadata)
......
...@@ -286,10 +286,6 @@ private[sql] class DefaultSource ...@@ -286,10 +286,6 @@ private[sql] class DefaultSource
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
sqlContext.conf.getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP)) sqlContext.conf.getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP))
// Whole stage codegen (PhysicalRDD) is able to deal with batches directly
val returningBatch =
supportBatch(sqlContext, StructType(partitionSchema.fields ++ dataSchema.fields))
// Try to push down filters when filter push-down is enabled. // Try to push down filters when filter push-down is enabled.
val pushed = if (sqlContext.getConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key).toBoolean) { val pushed = if (sqlContext.getConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key).toBoolean) {
filters filters
...@@ -308,8 +304,11 @@ private[sql] class DefaultSource ...@@ -308,8 +304,11 @@ private[sql] class DefaultSource
// TODO: if you move this into the closure it reverts to the default values. // TODO: if you move this into the closure it reverts to the default values.
// If true, enable using the custom RecordReader for parquet. This only works for // If true, enable using the custom RecordReader for parquet. This only works for
// a subset of the types (no complex types). // a subset of the types (no complex types).
val enableVectorizedParquetReader: Boolean = sqlContext.conf.parquetVectorizedReaderEnabled && val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
dataSchema.forall(_.dataType.isInstanceOf[AtomicType]) val enableVectorizedReader: Boolean = sqlContext.conf.parquetVectorizedReaderEnabled &&
resultSchema.forall(_.dataType.isInstanceOf[AtomicType])
// Whole stage codegen (PhysicalRDD) is able to deal with batches directly
val returningBatch = supportBatch(sqlContext, resultSchema)
(file: PartitionedFile) => { (file: PartitionedFile) => {
assert(file.partitionValues.numFields == partitionSchema.size) assert(file.partitionValues.numFields == partitionSchema.size)
...@@ -329,7 +328,7 @@ private[sql] class DefaultSource ...@@ -329,7 +328,7 @@ private[sql] class DefaultSource
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
val hadoopAttemptContext = new TaskAttemptContextImpl(broadcastedConf.value.value, attemptId) val hadoopAttemptContext = new TaskAttemptContextImpl(broadcastedConf.value.value, attemptId)
val parquetReader = if (enableVectorizedParquetReader) { val parquetReader = if (enableVectorizedReader) {
val vectorizedReader = new VectorizedParquetRecordReader() val vectorizedReader = new VectorizedParquetRecordReader()
vectorizedReader.initialize(split, hadoopAttemptContext) vectorizedReader.initialize(split, hadoopAttemptContext)
logDebug(s"Appending $partitionSchema ${file.partitionValues}") logDebug(s"Appending $partitionSchema ${file.partitionValues}")
...@@ -356,7 +355,7 @@ private[sql] class DefaultSource ...@@ -356,7 +355,7 @@ private[sql] class DefaultSource
// UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
if (parquetReader.isInstanceOf[VectorizedParquetRecordReader] && if (parquetReader.isInstanceOf[VectorizedParquetRecordReader] &&
enableVectorizedParquetReader) { enableVectorizedReader) {
iter.asInstanceOf[Iterator[InternalRow]] iter.asInstanceOf[Iterator[InternalRow]]
} else { } else {
val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
......
...@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path ...@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.sql._ import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow
import org.apache.spark.sql.execution.BatchedDataSourceScanExec
import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStruct, NestedStructUDT} import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStruct, NestedStructUDT}
import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SharedSQLContext
...@@ -589,6 +590,30 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext ...@@ -589,6 +590,30 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext
checkAnswer(sqlContext.read.parquet(path), df) checkAnswer(sqlContext.read.parquet(path), df)
} }
} }
test("returning batch for wide table") {
withSQLConf("spark.sql.codegen.maxFields" -> "100") {
withTempPath { dir =>
val path = dir.getCanonicalPath
val df = sqlContext.range(100).select(Seq.tabulate(110) {i => ('id + i).as(s"c$i")} : _*)
df.write.mode(SaveMode.Overwrite).parquet(path)
// donot return batch, because whole stage codegen is disabled for wide table (>200 columns)
val df2 = sqlContext.read.parquet(path)
assert(df2.queryExecution.sparkPlan.find(_.isInstanceOf[BatchedDataSourceScanExec]).isEmpty,
"Should not return batch")
checkAnswer(df2, df)
// return batch
val columns = Seq.tabulate(90) {i => s"c$i"}
val df3 = df2.selectExpr(columns : _*)
assert(
df3.queryExecution.sparkPlan.find(_.isInstanceOf[BatchedDataSourceScanExec]).isDefined,
"Should not return batch")
checkAnswer(df3, df.selectExpr(columns : _*))
}
}
}
} }
object TestingUDT { object TestingUDT {
......
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