-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-23731][SQL] Make FileSourceScanExec canonicalizable after being (de)serialized #21815
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -166,10 +166,12 @@ case class FileSourceScanExec( | |
| override val tableIdentifier: Option[TableIdentifier]) | ||
| extends DataSourceScanExec with ColumnarBatchScan { | ||
|
|
||
| override val supportsBatch: Boolean = relation.fileFormat.supportBatch( | ||
| // Note that some vals referring the file-based relation are lazy intentionally | ||
| // so that this plan can be canonicalized on executor side too. See SPARK-23731. | ||
| override lazy val supportsBatch: Boolean = relation.fileFormat.supportBatch( | ||
| relation.sparkSession, StructType.fromAttributes(output)) | ||
|
|
||
| override val needsUnsafeRowConversion: Boolean = { | ||
| override lazy val needsUnsafeRowConversion: Boolean = { | ||
| if (relation.fileFormat.isInstanceOf[ParquetSource]) { | ||
| SparkSession.getActiveSession.get.sessionState.conf.parquetVectorizedReaderEnabled | ||
| } else { | ||
|
|
@@ -199,7 +201,7 @@ case class FileSourceScanExec( | |
| ret | ||
| } | ||
|
|
||
| override val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = { | ||
| override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That happens on the driver so no need for the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it'd be computed anyway, though, when we create a new |
||
| val bucketSpec = if (relation.sparkSession.sessionState.conf.bucketingEnabled) { | ||
| relation.bucketSpec | ||
| } else { | ||
|
|
@@ -270,7 +272,7 @@ case class FileSourceScanExec( | |
| private val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter) | ||
| logInfo(s"Pushed Filters: ${pushedDownFilters.mkString(",")}") | ||
|
|
||
| override val metadata: Map[String, String] = { | ||
| override lazy val metadata: Map[String, String] = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's driver-only too, isn't it? Why is this
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it can be in executor side actually:
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ouch. I'd have never thought about any code with |
||
| def seqToString(seq: Seq[Any]) = seq.mkString("[", ", ", "]") | ||
| val location = relation.location | ||
| val locationDesc = | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -47,17 +47,15 @@ import org.apache.spark.util.ThreadUtils | |
| abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializable { | ||
|
|
||
| /** | ||
| * A handle to the SQL Context that was used to create this plan. Since many operators need | ||
| * A handle to the SQL Context that was used to create this plan. Since many operators need | ||
| * access to the sqlContext for RDD operations or configuration this field is automatically | ||
| * populated by the query planning infrastructure. | ||
| */ | ||
| @transient | ||
| final val sqlContext = SparkSession.getActiveSession.map(_.sqlContext).orNull | ||
| @transient final val sqlContext = SparkSession.getActiveSession.map(_.sqlContext).orNull | ||
|
|
||
| protected def sparkContext = sqlContext.sparkContext | ||
|
|
||
| // sqlContext will be null when SparkPlan nodes are created without the active sessions. | ||
| // So far, this only happens in the test cases. | ||
| val subexpressionEliminationEnabled: Boolean = if (sqlContext != null) { | ||
| sqlContext.conf.subexpressionEliminationEnabled | ||
| } else { | ||
|
|
@@ -69,7 +67,9 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ | |
|
|
||
| /** Overridden make copy also propagates sqlContext to copied plan. */ | ||
| override def makeCopy(newArgs: Array[AnyRef]): SparkPlan = { | ||
| SparkSession.setActiveSession(sqlContext.sparkSession) | ||
| if (sqlContext != null) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. just curious, why wasn't the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Because it looks failed ahead. Once we go with
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see, thanks, I wondered because it seems a more generic issue, easier to happen, but probably we never met it as all the trials included |
||
| SparkSession.setActiveSession(sqlContext.sparkSession) | ||
| } | ||
| super.makeCopy(newArgs) | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since you mentioned
SparkSession, that line caught my attention where the activeSparkSessionis accessed usingSparkSession.getActiveSession.getnotrelation.sparkSessionas is the case for other places. I think that's something worth considering changing since we're at it.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's leave this out of this PR's scope. That's more like making the plan workable whereas this PR targets the plan can be canonicalized.