Skip to content

Commit b488836

Browse files
committed
Address comments
1 parent 22c7986 commit b488836

File tree

2 files changed

+5
-3
lines changed

2 files changed

+5
-3
lines changed

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -166,6 +166,8 @@ case class FileSourceScanExec(
166166
override val tableIdentifier: Option[TableIdentifier])
167167
extends DataSourceScanExec with ColumnarBatchScan {
168168

169+
// Note that some vals referring the file-based relation are lazy intentionally
170+
// so that this plan can be canonicalized on executor side too. See SPARK-23731.
169171
override lazy val supportsBatch: Boolean = relation.fileFormat.supportBatch(
170172
relation.sparkSession, StructType.fromAttributes(output))
171173

sql/core/src/test/scala/org/apache/spark/sql/execution/FileSourceScanExecSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,12 +20,12 @@ package org.apache.spark.sql.execution
2020
import org.apache.spark.sql.test.SharedSQLContext
2121

2222
class FileSourceScanExecSuite extends SharedSQLContext {
23-
test("FileSourceScanExec should be canonicalizable in executor side") {
23+
test("FileSourceScanExec should be canonicalizable on executor side") {
2424
withTempPath { path =>
25-
spark.range(1).toDF().write.parquet(path.getAbsolutePath)
25+
spark.range(1).write.parquet(path.getAbsolutePath)
2626
val df = spark.read.parquet(path.getAbsolutePath)
2727
val fileSourceScanExec =
28-
df.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get
28+
df.queryExecution.sparkPlan.collectFirst { case p: FileSourceScanExec => p }.get
2929
try {
3030
spark.range(1).foreach(_ => fileSourceScanExec.canonicalized)
3131
} catch {

0 commit comments

Comments
 (0)